You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by st...@apache.org on 2022/09/27 13:28:37 UTC

[impala] branch master updated: IMPALA-11420: Support CREATE/ALTER VIEW SET/UNSET TBLPROPERTIES

This is an automated email from the ASF dual-hosted git repository.

stigahuang pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/impala.git


The following commit(s) were added to refs/heads/master by this push:
     new a377662e9 IMPALA-11420: Support CREATE/ALTER VIEW SET/UNSET TBLPROPERTIES
a377662e9 is described below

commit a377662e9427acc353210fc0883f856704ef9c24
Author: xiabaike <xi...@163.com>
AuthorDate: Mon Sep 5 11:07:56 2022 +0000

    IMPALA-11420: Support CREATE/ALTER VIEW SET/UNSET TBLPROPERTIES
    
    Add TBLPROPERTIES support to the view, here are some examples:
      CREATE VIEW [IF NOT EXISTS] [database_name.]view_name
        [(column_name [COMMENT 'column_comment'][, ...])]
        [COMMENT 'view_comment']
        [TBLPROPERTIES (property_name = property_value, ...)]
        AS select_statement;
    
      ALTER VIEW [database_name.]view_name SET TBLPROPERTIES
        (property_name = property_value, ...);
    
      ALTER VIEW [database_name.]view_name UNSET TBLPROPERTIES
        (property_name, ...);
    
    Change-Id: I8d05bb4ec1f70f5387bb21fbe23f62c05941af18
    Reviewed-on: http://gerrit.cloudera.org:8080/18940
    Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 common/thrift/JniCatalog.thrift                    |  5 ++
 fe/src/main/cup/sql-parser.cup                     | 17 +++-
 .../impala/analysis/AlterViewSetTblProperties.java | 93 ++++++++++++++++++++
 .../org/apache/impala/analysis/AlterViewStmt.java  |  2 +-
 .../analysis/AlterViewUnSetTblProperties.java      | 99 ++++++++++++++++++++++
 .../java/org/apache/impala/analysis/Analyzer.java  |  1 +
 .../impala/analysis/CreateOrAlterViewStmtBase.java | 22 ++++-
 .../org/apache/impala/analysis/CreateViewStmt.java | 11 ++-
 .../org/apache/impala/analysis/ToSqlUtils.java     |  2 +-
 .../apache/impala/service/CatalogOpExecutor.java   | 63 ++++++++++++--
 .../org/apache/impala/analysis/AnalyzeDDLTest.java | 38 ++++++++-
 .../org/apache/impala/analysis/ParserTest.java     | 21 +++++
 .../java/org/apache/impala/analysis/ToSqlTest.java | 12 +++
 .../queries/QueryTest/views-ddl.test               | 70 ++++++++++++++-
 14 files changed, 441 insertions(+), 15 deletions(-)

diff --git a/common/thrift/JniCatalog.thrift b/common/thrift/JniCatalog.thrift
old mode 100644
new mode 100755
index 7f68dfaf3..dd87ba72b
--- a/common/thrift/JniCatalog.thrift
+++ b/common/thrift/JniCatalog.thrift
@@ -115,6 +115,8 @@ enum TAlterTableType {
   UNSET_TBL_PROPERTIES = 17
   SET_PARTITION_SPEC = 18
   EXECUTE = 19
+  SET_VIEW_PROPERTIES = 20
+  UNSET_VIEW_PROPERTIES = 21
 }
 
 // Parameters of CREATE DATABASE commands
@@ -617,6 +619,9 @@ struct TCreateOrAlterViewParams {
   // The server name for security privileges when authorization is enabled.
   // TODO: Need to cleanup:IMPALA-7553
   8: optional string server_name
+
+  // Tblproperties of the view
+  9: optional map<string, string> tblproperties
 }
 
 // Parameters of a COMPUTE STATS command
diff --git a/fe/src/main/cup/sql-parser.cup b/fe/src/main/cup/sql-parser.cup
old mode 100644
new mode 100755
index b7f85f356..9656ceb40
--- a/fe/src/main/cup/sql-parser.cup
+++ b/fe/src/main/cup/sql-parser.cup
@@ -2197,9 +2197,12 @@ block_size_val ::=
 
 create_view_stmt ::=
   KW_CREATE KW_VIEW if_not_exists_val:if_not_exists table_name:view_name
-  view_column_defs:col_defs opt_comment_val:comment KW_AS query_stmt:view_def
+  view_column_defs:col_defs opt_comment_val:comment
+  tbl_properties:tbl_props
+  KW_AS query_stmt:view_def
   {:
-    RESULT = new CreateViewStmt(if_not_exists, view_name, col_defs, comment, view_def);
+    RESULT = new CreateViewStmt(if_not_exists, view_name, col_defs, comment, tbl_props,
+         view_def);
   :}
   ;
 
@@ -2340,6 +2343,16 @@ alter_view_stmt ::=
     parser.checkIdentKeyword("OWNER", owner_id);
     RESULT = new AlterViewSetOwnerStmt(table, new Owner(TOwnerType.ROLE, role));
   :}
+  | KW_ALTER KW_VIEW table_name:table KW_SET
+    tbl_properties:tbl_props
+  {:
+    RESULT = new AlterViewSetTblProperties(table, tbl_props);
+  :}
+  | KW_ALTER KW_VIEW table_name:table KW_UNSET
+    KW_TBLPROPERTIES if_exists_val:if_exists LPAREN properties_key_list:property_keys RPAREN
+  {:
+    RESULT = new AlterViewUnSetTblProperties(table, if_exists, property_keys);
+  :}
   ;
 
 cascade_val ::=
diff --git a/fe/src/main/java/org/apache/impala/analysis/AlterViewSetTblProperties.java b/fe/src/main/java/org/apache/impala/analysis/AlterViewSetTblProperties.java
new file mode 100755
index 000000000..85408ca05
--- /dev/null
+++ b/fe/src/main/java/org/apache/impala/analysis/AlterViewSetTblProperties.java
@@ -0,0 +1,93 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package org.apache.impala.analysis;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
+import org.apache.impala.authorization.Privilege;
+import org.apache.impala.catalog.FeView;
+import org.apache.impala.catalog.MaterializedViewHdfsTable;
+import org.apache.impala.common.AnalysisException;
+import org.apache.impala.thrift.TAlterTableParams;
+import org.apache.impala.thrift.TAlterTableSetTblPropertiesParams;
+import org.apache.impala.thrift.TAlterTableType;
+import org.apache.impala.thrift.TTableName;
+import org.apache.impala.thrift.TTablePropertyType;
+import org.apache.impala.util.MetaStoreUtil;
+
+import java.util.Map;
+
+/**
+ * Represents an ALTER VIEW SET TBLPROPERTIES ('p1'='v1', ...) statement.
+ */
+public class AlterViewSetTblProperties extends AlterTableSetStmt {
+
+  private final Map<String, String> tblProperties_;
+
+  public AlterViewSetTblProperties(TableName tableName,
+      Map<String, String> tblProperties) {
+    super(tableName, null);
+    Preconditions.checkNotNull(tblProperties);
+    tblProperties_ = tblProperties;
+    CreateTableStmt.unescapeProperties(tblProperties_);
+  }
+
+  @Override
+  public TAlterTableParams toThrift() {
+    TAlterTableParams params = new TAlterTableParams();
+    params.setTable_name(new TTableName(getDb(), getTbl()));
+    params.setAlter_type(TAlterTableType.SET_VIEW_PROPERTIES);
+    TAlterTableSetTblPropertiesParams tblPropertyParams =
+        new TAlterTableSetTblPropertiesParams();
+    tblPropertyParams.setTarget(TTablePropertyType.TBL_PROPERTY);
+    tblPropertyParams.setProperties(tblProperties_);
+    params.setSet_tbl_properties_params(tblPropertyParams);
+    return params;
+  }
+
+  @Override
+  public void analyze(Analyzer analyzer) throws AnalysisException {
+    MetaStoreUtil.checkShortPropertyMap("Property", tblProperties_);
+
+    table_ = analyzer.getTable(tableName_, Privilege.ALTER);
+    Preconditions.checkNotNull(table_);
+    if (table_ instanceof MaterializedViewHdfsTable) {
+      throw new AnalysisException(String.format(
+          "ALTER VIEW not allowed on a materialized view: %s", tableName_));
+    } else if (!(table_ instanceof FeView)) {
+      throw new AnalysisException(String.format(
+          "ALTER VIEW not allowed on a table: %s", tableName_));
+    }
+
+    if (tblProperties_.containsKey(hive_metastoreConstants.META_TABLE_STORAGE)) {
+      throw new AnalysisException(String.format("Changing the '%s' view property is " +
+          "not supported to protect against metadata corruption.",
+          hive_metastoreConstants.META_TABLE_STORAGE));
+    }
+  }
+
+  @Override
+  public String toSql(ToSqlOptions options) {
+    StringBuilder sb = new StringBuilder();
+    sb.append("ALTER VIEW ");
+    if (tableName_.getDb() != null) sb.append(tableName_.getDb() + ".");
+    sb.append(tableName_.getTbl());
+    sb.append(" SET TBLPROPERTIES " + ToSqlUtils.propertyMapToSql(tblProperties_));
+    return sb.toString();
+  }
+}
\ No newline at end of file
diff --git a/fe/src/main/java/org/apache/impala/analysis/AlterViewStmt.java b/fe/src/main/java/org/apache/impala/analysis/AlterViewStmt.java
old mode 100644
new mode 100755
index f89dba761..fd6668a7d
--- a/fe/src/main/java/org/apache/impala/analysis/AlterViewStmt.java
+++ b/fe/src/main/java/org/apache/impala/analysis/AlterViewStmt.java
@@ -38,7 +38,7 @@ import com.google.common.base.Preconditions;
 public class AlterViewStmt extends CreateOrAlterViewStmtBase {
   public AlterViewStmt(
       TableName tableName, List<ColumnDef> columnDefs, QueryStmt viewDefStmt) {
-    super(false, tableName, columnDefs, null, viewDefStmt);
+    super(false, tableName, columnDefs, null, null, viewDefStmt);
   }
 
   @Override
diff --git a/fe/src/main/java/org/apache/impala/analysis/AlterViewUnSetTblProperties.java b/fe/src/main/java/org/apache/impala/analysis/AlterViewUnSetTblProperties.java
new file mode 100755
index 000000000..ea6cb4617
--- /dev/null
+++ b/fe/src/main/java/org/apache/impala/analysis/AlterViewUnSetTblProperties.java
@@ -0,0 +1,99 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package org.apache.impala.analysis;
+
+import com.google.common.base.Joiner;
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
+import org.apache.impala.authorization.Privilege;
+import org.apache.impala.catalog.FeView;
+import org.apache.impala.catalog.MaterializedViewHdfsTable;
+import org.apache.impala.common.AnalysisException;
+import org.apache.impala.thrift.TAlterTableParams;
+import org.apache.impala.thrift.TAlterTableType;
+import org.apache.impala.thrift.TAlterTableUnSetTblPropertiesParams;
+import org.apache.impala.thrift.TTableName;
+import org.apache.impala.thrift.TTablePropertyType;
+import org.apache.impala.util.MetaStoreUtil;
+
+import java.util.List;
+
+/**
+ * Represents an ALTER VIEW UNSET TBLPROPERTIES ('p1','p2' ...) statement.
+ */
+public class AlterViewUnSetTblProperties extends AlterTableStmt {
+  private final List<String> tblPropertyKeys_;
+  private final boolean ifExists_;
+
+  public AlterViewUnSetTblProperties(TableName tableName, boolean ifExist,
+      List<String> tblPropertyKeys) {
+    super(tableName);
+    Preconditions.checkNotNull(tblPropertyKeys);
+    tblPropertyKeys_ = tblPropertyKeys;
+    ifExists_ = ifExist;
+  }
+
+  @Override
+  public TAlterTableParams toThrift() {
+    TAlterTableParams params = new TAlterTableParams();
+    params.setTable_name(new TTableName(getDb(), getTbl()));
+    params.setAlter_type(TAlterTableType.UNSET_VIEW_PROPERTIES);
+    TAlterTableUnSetTblPropertiesParams tblUnsetPropertyParams =
+        new TAlterTableUnSetTblPropertiesParams();
+    tblUnsetPropertyParams.setTarget(TTablePropertyType.TBL_PROPERTY);
+    tblUnsetPropertyParams.setProperty_keys(tblPropertyKeys_);
+    tblUnsetPropertyParams.setIf_exists(ifExists_);
+    params.setUnset_tbl_properties_params(tblUnsetPropertyParams);
+    return params;
+  }
+
+  @Override
+  public void analyze(Analyzer analyzer) throws AnalysisException {
+    for (String property : tblPropertyKeys_) {
+      MetaStoreUtil.checkShortProperty("Property key", property,
+          MetaStoreUtil.MAX_PROPERTY_KEY_LENGTH);
+    }
+
+    table_ = analyzer.getTable(tableName_, Privilege.ALTER);
+    Preconditions.checkNotNull(table_);
+    if (table_ instanceof MaterializedViewHdfsTable) {
+      throw new AnalysisException(String.format(
+          "ALTER VIEW not allowed on a materialized view: %s", tableName_));
+    } else if (!(table_ instanceof FeView)) {
+      throw new AnalysisException(String.format(
+          "ALTER VIEW not allowed on a table: %s", tableName_));
+    }
+
+    if (tblPropertyKeys_.contains(hive_metastoreConstants.META_TABLE_STORAGE)) {
+      throw new AnalysisException(String.format("Changing the '%s' view property is " +
+          "not supported to protect against metadata corruption.",
+          hive_metastoreConstants.META_TABLE_STORAGE));
+    }
+  }
+
+  @Override
+  public String toSql(ToSqlOptions options) {
+    StringBuilder sb = new StringBuilder();
+    sb.append("ALTER VIEW ");
+    if (tableName_.getDb() != null) sb.append(tableName_.getDb() + ".");
+    sb.append(tableName_.getTbl());
+    sb.append(" UNSET TBLPROPERTIES ('" +
+        Joiner.on("', '").join(tblPropertyKeys_) + "')");
+    return sb.toString();
+  }
+}
\ No newline at end of file
diff --git a/fe/src/main/java/org/apache/impala/analysis/Analyzer.java b/fe/src/main/java/org/apache/impala/analysis/Analyzer.java
index 91f27b470..831d35d73 100644
--- a/fe/src/main/java/org/apache/impala/analysis/Analyzer.java
+++ b/fe/src/main/java/org/apache/impala/analysis/Analyzer.java
@@ -157,6 +157,7 @@ public class Analyzer {
   public final static String DB_ALREADY_EXISTS_ERROR_MSG = "Database already exists: ";
   public final static String TBL_DOES_NOT_EXIST_ERROR_MSG = "Table does not exist: ";
   public final static String TBL_ALREADY_EXISTS_ERROR_MSG = "Table already exists: ";
+  public final static String VIEW_ALREADY_EXISTS_ERROR_MSG = "View already exists: ";
   public final static String FN_DOES_NOT_EXIST_ERROR_MSG = "Function does not exist: ";
   public final static String FN_ALREADY_EXISTS_ERROR_MSG = "Function already exists: ";
   public final static String DATA_SRC_DOES_NOT_EXIST_ERROR_MSG =
diff --git a/fe/src/main/java/org/apache/impala/analysis/CreateOrAlterViewStmtBase.java b/fe/src/main/java/org/apache/impala/analysis/CreateOrAlterViewStmtBase.java
old mode 100644
new mode 100755
index 5c064af2d..660d0930f
--- a/fe/src/main/java/org/apache/impala/analysis/CreateOrAlterViewStmtBase.java
+++ b/fe/src/main/java/org/apache/impala/analysis/CreateOrAlterViewStmtBase.java
@@ -18,8 +18,10 @@
 package org.apache.impala.analysis;
 
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.HashSet;
 import java.util.List;
+import java.util.Map;
 import java.util.Set;
 
 import org.apache.impala.analysis.ColumnLineageGraph.ColumnLabel;
@@ -45,6 +47,7 @@ public abstract class CreateOrAlterViewStmtBase extends StatementBase {
   protected final List<ColumnDef> columnDefs_;
   protected final String comment_;
   protected final QueryStmt viewDefStmt_;
+  protected Map<String, String> tblPropertyMap_;
 
   // Set during analysis
   protected String dbName_;
@@ -75,13 +78,15 @@ public abstract class CreateOrAlterViewStmtBase extends StatementBase {
   protected List<ColumnDef> finalColDefs_;
 
   public CreateOrAlterViewStmtBase(boolean ifNotExists, TableName tableName,
-      List<ColumnDef> columnDefs, String comment, QueryStmt viewDefStmt) {
+      List<ColumnDef> columnDefs, String comment, Map<String, String> tblPropertyMap,
+      QueryStmt viewDefStmt) {
     Preconditions.checkNotNull(tableName);
     Preconditions.checkNotNull(viewDefStmt);
     this.ifNotExists_ = ifNotExists;
     this.tableName_ = tableName;
     this.columnDefs_ = columnDefs;
     this.comment_ = comment;
+    this.tblPropertyMap_ = tblPropertyMap;
     this.viewDefStmt_ = viewDefStmt;
   }
 
@@ -197,6 +202,7 @@ public abstract class CreateOrAlterViewStmtBase extends StatementBase {
     params.setExpanded_view_def(inlineViewDef_);
     params.setServer_name(serverName_);
     params.setComment(comment_);
+    params.setTblproperties(getTblPropertyMap());
     return params;
   }
 
@@ -230,6 +236,20 @@ public abstract class CreateOrAlterViewStmtBase extends StatementBase {
     return Joiner.on(", ").join(columnNames);
   }
 
+  protected Map<String, String> getTblPropertyMap() {
+    if (tblPropertyMap_ == null || tblPropertyMap_.size() == 0) {
+      return Collections.emptyMap();
+    }
+    return tblPropertyMap_;
+  }
+
+  protected String getTblProperties() {
+    if (tblPropertyMap_ == null) {
+      return null;
+    }
+    return ToSqlUtils.propertyMapToSql(tblPropertyMap_);
+  }
+
   public boolean getIfNotExists() { return ifNotExists_; }
   public String getInlineViewDef() { return inlineViewDef_; }
   public String getTbl() { return tableName_.getTbl(); }
diff --git a/fe/src/main/java/org/apache/impala/analysis/CreateViewStmt.java b/fe/src/main/java/org/apache/impala/analysis/CreateViewStmt.java
old mode 100644
new mode 100755
index a387022fa..a133e53c0
--- a/fe/src/main/java/org/apache/impala/analysis/CreateViewStmt.java
+++ b/fe/src/main/java/org/apache/impala/analysis/CreateViewStmt.java
@@ -18,6 +18,7 @@
 package org.apache.impala.analysis;
 
 import java.util.List;
+import java.util.Map;
 
 import org.apache.impala.authorization.Privilege;
 import org.apache.impala.common.AnalysisException;
@@ -33,8 +34,9 @@ import com.google.common.base.Preconditions;
  */
 public class CreateViewStmt extends CreateOrAlterViewStmtBase {
   public CreateViewStmt(boolean ifNotExists, TableName tableName,
-      List<ColumnDef> columnDefs, String comment, QueryStmt viewDefStmt) {
-    super(ifNotExists, tableName, columnDefs, comment, viewDefStmt);
+      List<ColumnDef> columnDefs, String comment, Map<String, String> tblpropertiesMap,
+      QueryStmt viewDefStmt) {
+    super(ifNotExists, tableName, columnDefs, comment, tblpropertiesMap, viewDefStmt);
   }
 
   @Override
@@ -56,7 +58,7 @@ public class CreateViewStmt extends CreateOrAlterViewStmtBase {
 
     if (analyzer.dbContainsTable(dbName_, tableName_.getTbl(), Privilege.CREATE) &&
         !ifNotExists_) {
-      throw new AnalysisException(Analyzer.TBL_ALREADY_EXISTS_ERROR_MSG +
+      throw new AnalysisException(Analyzer.VIEW_ALREADY_EXISTS_ERROR_MSG +
           String.format("%s.%s", dbName_, tableName_.getTbl()));
     }
     analyzer.addAccessEvent(new TAccessEvent(dbName_ + "." + tableName_.getTbl(),
@@ -76,6 +78,9 @@ public class CreateViewStmt extends CreateOrAlterViewStmtBase {
     if (tableName_.getDb() != null) sb.append(tableName_.getDb() + ".");
     sb.append(tableName_.getTbl());
     if (columnDefs_ != null) sb.append("(" + getColumnNames() + ")");
+    if (tblPropertyMap_ != null && !tblPropertyMap_.isEmpty()) {
+      sb.append(" TBLPROPERTIES " + getTblProperties());
+    }
     sb.append(" AS ");
     sb.append(viewDefStmt_.toSql(options));
     return sb.toString();
diff --git a/fe/src/main/java/org/apache/impala/analysis/ToSqlUtils.java b/fe/src/main/java/org/apache/impala/analysis/ToSqlUtils.java
old mode 100644
new mode 100755
index e5f37099f..e261c310b
--- a/fe/src/main/java/org/apache/impala/analysis/ToSqlUtils.java
+++ b/fe/src/main/java/org/apache/impala/analysis/ToSqlUtils.java
@@ -618,7 +618,7 @@ public class ToSqlUtils {
     return sb.toString();
   }
 
-  private static String propertyMapToSql(Map<String, String> propertyMap) {
+  public static String propertyMapToSql(Map<String, String> propertyMap) {
     // Sort entries on the key to ensure output is deterministic for tests (IMPALA-5757).
     List<Entry<String, String>> mapEntries = Lists.newArrayList(propertyMap.entrySet());
     Collections.sort(mapEntries, new Comparator<Entry<String, String>>() {
diff --git a/fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java b/fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java
old mode 100644
new mode 100755
index 6a9e833e1..403b204b5
--- a/fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java
+++ b/fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java
@@ -336,6 +336,10 @@ public class CatalogOpExecutor {
       "Please use the following syntax if not sure whether the property existed" +
       " or not:\nALTER TABLE tableName UNSET (TBLPROPERTIES|SERDEPROPERTIES) IF EXISTS" +
       " (key1, key2, ...)\n";
+  private final static String ALTER_VIEW_UNSET_NON_EXIST_PROPERTY =
+      "Please use the following syntax if not sure whether the property existed" +
+      " or not:\nALTER VIEW viewName UNSET TBLPROPERTIES IF EXISTS" +
+      " (key1, key2, ...)\n";
 
   // Table default capabilities
   private static final String ACIDINSERTONLY_CAPABILITIES =
@@ -1159,6 +1163,16 @@ public class CatalogOpExecutor {
             responseSummaryMsg = "Updated table.";
           }
           break;
+        case SET_VIEW_PROPERTIES:
+          alterViewSetTblProperties(tbl, params.getSet_tbl_properties_params());
+          reloadTableSchema = true;
+          responseSummaryMsg = "Updated view.";
+          break;
+        case UNSET_VIEW_PROPERTIES:
+          alterViewUnSetTblProperties(tbl, params.getUnset_tbl_properties_params());
+          reloadTableSchema = true;
+          responseSummaryMsg = "Updated view.";
+          break;
         case UPDATE_STATS:
           Preconditions.checkState(params.isSetUpdate_stats_params());
           Reference<Long> numUpdatedColumns = new Reference<>(0L);
@@ -3902,6 +3916,9 @@ public class CatalogOpExecutor {
     if (params.isSetComment() && params.getComment() != null) {
       view.getParameters().put("comment", params.getComment());
     }
+    if (params.getTblproperties() != null && params.getTblpropertiesSize() != 0) {
+      view.getParameters().putAll(params.getTblproperties());
+    }
     StorageDescriptor sd = new StorageDescriptor();
     // Add all the columns to a new storage descriptor.
     sd.setCols(buildFieldSchemaList(params.getColumns()));
@@ -3921,6 +3938,9 @@ public class CatalogOpExecutor {
     if (params.isSetComment() && params.getComment() != null) {
       view.getParameters().put("comment", params.getComment());
     }
+    if (params.getTblproperties() != null && params.getTblpropertiesSize() != 0) {
+      view.getParameters().putAll(params.getTblproperties());
+    }
     // Add all the columns to a new storage descriptor.
     view.getSd().setCols(buildFieldSchemaList(params.getColumns()));
   }
@@ -5305,8 +5325,8 @@ public class CatalogOpExecutor {
             throw new UnsupportedOperationException(
                 "Unknown target TTablePropertyType: " + params.getTarget());
         }
-        removeKeys(removeProperties, ifExists, keys,
-            "partition " + partition.getPartitionName());
+        removeKeys(removeProperties, ifExists, keys, "partition " +
+            partition.getPartitionName(), ALTER_TBL_UNSET_NON_EXIST_PROPERTY);
         modifiedParts.add(partBuilder);
       }
       try {
@@ -5332,7 +5352,8 @@ public class CatalogOpExecutor {
           throw new UnsupportedOperationException(
               "Unknown target TTablePropertyType: " + params.getTarget());
       }
-      removeKeys(removeProperties, ifExists, keys, "table " + tbl.getFullName());
+      removeKeys(removeProperties, ifExists, keys,
+          "table " + tbl.getFullName(), ALTER_TBL_UNSET_NON_EXIST_PROPERTY);
       // Validate that the new table properties are valid and that
       // the Kudu table is accessible.
       if (KuduTable.isKuduTable(msTbl)) {
@@ -5342,8 +5363,40 @@ public class CatalogOpExecutor {
     }
   }
 
+  /**
+   * Appends to the view property metadata for the given view, replacing
+   * the values of any keys that already exist.
+   */
+  private void alterViewSetTblProperties(Table tbl,
+      TAlterTableSetTblPropertiesParams params) throws ImpalaException {
+    Preconditions.checkState(tbl.isWriteLockedByCurrentThread());
+    Map<String, String> properties = params.getProperties();
+    Preconditions.checkNotNull(properties);
+
+    // Alter view params.
+    org.apache.hadoop.hive.metastore.api.Table msTbl =
+        tbl.getMetaStoreTable().deepCopy();
+    msTbl.getParameters().putAll(properties);
+    applyAlterTable(msTbl);
+  }
+
+  private void alterViewUnSetTblProperties(Table tbl,
+      TAlterTableUnSetTblPropertiesParams params) throws ImpalaException {
+    Preconditions.checkState(tbl.isWriteLockedByCurrentThread());
+    List<String> removeProperties = params.getProperty_keys();
+    boolean ifExists = params.isIf_exists();
+    Preconditions.checkNotNull(removeProperties);
+    // Alter view params.
+    org.apache.hadoop.hive.metastore.api.Table msTbl =
+            tbl.getMetaStoreTable().deepCopy();
+    Set<String> keys = msTbl.getParameters().keySet();
+    removeKeys(removeProperties, ifExists, keys,
+        "view " + tbl.getFullName(), ALTER_VIEW_UNSET_NON_EXIST_PROPERTY);
+    applyAlterTable(msTbl);
+  }
+
   private void removeKeys(List<String> removeProperties, boolean ifExists,
-      Set<String> keys, String fullName) throws CatalogException {
+      Set<String> keys, String fullName, String excepInfo) throws CatalogException {
     if (ifExists || keys.containsAll(removeProperties)) {
       keys.removeAll(removeProperties);
     } else {
@@ -5353,7 +5406,7 @@ public class CatalogOpExecutor {
           String.format("These properties do not exist for %s: %s.\n%s",
               fullName,
               String.join(",", removeCopy),
-              ALTER_TBL_UNSET_NON_EXIST_PROPERTY));
+              excepInfo));
     }
   }
 
diff --git a/fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java b/fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java
old mode 100644
new mode 100755
index 30dfb15be..49416a25e
--- a/fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java
+++ b/fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java
@@ -1496,6 +1496,34 @@ public class AnalyzeDDLTest extends FrontendTestBase {
         "column 'new_col'.");
   }
 
+  @Test
+  public void TestAlterViewSetTblProperties() throws AnalysisException {
+    AnalyzesOk("ALTER VIEW functional.alltypes_view SET TBLPROPERTIES " +
+        " ('pro1' = 'test1', 'pro2' = 'test2')");
+    AnalyzesOk("ALTER VIEW functional.alltypes_view UNSET TBLPROPERTIES " +
+        " ('pro1', 'pro2')");
+
+    AnalysisError("ALTER VIEW Foo.Bar SET TBLPROPERTIES " +
+        " ('pro1' = 'test1', 'pro2' = 'test2')",
+        "Database does not exist: Foo");
+    AnalysisError("ALTER VIEW Foo.Bar UNSET TBLPROPERTIES ('pro1', 'pro2')",
+        "Database does not exist: Foo");
+    AnalysisError("alter view functional_orc_def.mv1_alltypes_jointbl " +
+        "set TBLPROPERTIES('a' = 'a')",
+        "ALTER VIEW not allowed on a materialized view: " +
+        "functional_orc_def.mv1_alltypes_jointbl");
+    AnalysisError("alter view functional.alltypes " +
+        "set TBLPROPERTIES('a' = 'a')",
+        "ALTER VIEW not allowed on a table: functional.alltypes");
+    AnalysisError("alter view functional_orc_def.mv1_alltypes_jointbl " +
+        "unset TBLPROPERTIES('a')",
+        "ALTER VIEW not allowed on a materialized view: " +
+        "functional_orc_def.mv1_alltypes_jointbl");
+    AnalysisError("alter view functional.alltypes " +
+        "unset TBLPROPERTIES('a')",
+        "ALTER VIEW not allowed on a table: functional.alltypes");
+  }
+
   @Test
   public void TestAlterViewRename() throws AnalysisException {
     AnalyzesOk("alter view functional.alltypes_view rename to new_view");
@@ -3068,7 +3096,7 @@ public class AnalyzeDDLTest extends FrontendTestBase {
     // Table/view already exists.
     AnalysisError("create view functional.alltypes as " +
         "select * from functional.alltypessmall ",
-        "Table already exists: functional.alltypes");
+        "View already exists: functional.alltypes");
     // Target database does not exist.
     AnalysisError("create view wrongdb.test as " +
         "select * from functional.alltypessmall ",
@@ -3109,6 +3137,14 @@ public class AnalyzeDDLTest extends FrontendTestBase {
     AnalysisError("create view v as select null as new_col",
         "Unable to infer the column type for column 'new_col'. Use cast() to " +
             "explicitly specify the column type for column 'new_col'.");
+
+    AnalyzesOk("create view v_tblproperties TBLPROPERTIES ('a' = 'a')" +
+        "as select cast(null as int) as new_col");
+    AnalyzesOk("create view v_tblproperties TBLPROPERTIES ('a' = 'a', 'b' = 'b')" +
+        "as select cast(null as int) as new_col");
+    AnalysisError("create view functional.view_view TBLPROPERTIES ('a' = 'a')" +
+        "as select cast(null as int) as new_col",
+        "View already exists: functional.view_view");
   }
 
   @Test
diff --git a/fe/src/test/java/org/apache/impala/analysis/ParserTest.java b/fe/src/test/java/org/apache/impala/analysis/ParserTest.java
old mode 100644
new mode 100755
index 48b64ec31..db78a3d70
--- a/fe/src/test/java/org/apache/impala/analysis/ParserTest.java
+++ b/fe/src/test/java/org/apache/impala/analysis/ParserTest.java
@@ -3135,6 +3135,16 @@ public class ParserTest extends FrontendTestBase {
     // Mismatched number of columns in column definition and view definition parses ok.
     ParsesOk("CREATE VIEW Bar (x, y) AS SELECT 1, 2, 3");
 
+    ParsesOk("CREATE VIEW Bar (x, y, z) TBLPROPERTIES ('a' = 'b') AS SELECT 1, 2, 3");
+    ParsesOk("CREATE VIEW Bar (x, y, z) TBLPROPERTIES ('a' = 'b', 'c' = 'd')" +
+        " AS SELECT 1, 2, 3");
+    ParsesOk("CREATE VIEW Bar TBLPROPERTIES ('a' = 'b') AS VALUES(1, 2, 3)");
+    ParsesOk("CREATE VIEW Bar TBLPROPERTIES ('a' = 'b') AS SELECT 1, 2, 3");
+    ParsesOk("CREATE VIEW Bar TBLPROPERTIES ('a' = 'b', 'c' = 'd')" +
+        " AS SELECT 1, 2, 3");
+    ParsesOk("CREATE VIEW Foo.Bar COMMENT 'test' TBLPROPERTIES ('a' = 'b')" +
+        " AS SELECT a, b, c from t");
+
     // No view name.
     ParserError("CREATE VIEW AS SELECT c FROM t");
     // Missing AS keyword
@@ -3155,6 +3165,9 @@ public class ParserTest extends FrontendTestBase {
     ParserError("CREATE VIEW Foo.Bar (x) AS ALTER TABLE Foo COLUMNS (i int, s string)");
     ParserError("CREATE VIEW Foo.Bar (x) AS CREATE VIEW Foo.Bar AS SELECT 1");
     ParserError("CREATE VIEW Foo.Bar (x) AS ALTER VIEW Foo.Bar AS SELECT 1");
+
+    ParserError("CREATE VIEW Bar (x, y, z) TBLPROPERTIES () AS SELECT 1, 2, 3");
+    ParserError("CREATE VIEW Bar (x, y, z) TBLPROPERTIES (i int) AS SELECT 1, 2, 3");
   }
 
   @Test
@@ -3177,6 +3190,9 @@ public class ParserTest extends FrontendTestBase {
     // Mismatched number of columns in column definition and view definition parses ok.
     ParsesOk("ALTER VIEW Bar (x, y) AS SELECT 1, 2, 3");
 
+    ParsesOk("ALTER VIEW Foo.Bar SET TBLPROPERTIES ('pro1' = '1', 'pro2' = '2')");
+    ParsesOk("ALTER VIEW Foo.Bar UNSET TBLPROPERTIES ('pro1', 'pro2')");
+
     // Must be ALTER VIEW not ALTER TABLE.
     ParserError("ALTER TABLE Foo.Bar AS SELECT 1, 2, 3");
     // Missing view name.
@@ -3200,6 +3216,11 @@ public class ParserTest extends FrontendTestBase {
     ParserError("ALTER VIEW Foo.Bar AS ALTER TABLE Foo COLUMNS (i int, s string)");
     ParserError("ALTER VIEW Foo.Bar AS CREATE VIEW Foo.Bar AS SELECT 1, 2, 3");
     ParserError("ALTER VIEW Foo.Bar AS ALTER VIEW Foo.Bar AS SELECT 1, 2, 3");
+
+    ParserError("ALTER VIEW Foo.Bar SET TBLPROPERTIES ()");
+    ParserError("ALTER VIEW Foo.Bar SET TBLPROPERTIES (int COMMENT 'x')");
+    ParserError("ALTER VIEW Foo.Bar UNSET TBLPROPERTIES ()");
+    ParserError("ALTER VIEW Foo.Bar UNSET TBLPROPERTIES (int COMMENT 'x')");
   }
 
   @Test
diff --git a/fe/src/test/java/org/apache/impala/analysis/ToSqlTest.java b/fe/src/test/java/org/apache/impala/analysis/ToSqlTest.java
index 1ffe71114..83873b302 100644
--- a/fe/src/test/java/org/apache/impala/analysis/ToSqlTest.java
+++ b/fe/src/test/java/org/apache/impala/analysis/ToSqlTest.java
@@ -503,6 +503,12 @@ public class ToSqlTest extends FrontendTestBase {
         "CREATE VIEW test_view_with_subquery AS " +
         "SELECT * FROM functional.alltypestiny t WHERE EXISTS " +
         "(SELECT * FROM functional.alltypessmall s WHERE s.id = t.id)");
+    testToSql(
+      "create view test_properties tblproperties ('a'='aa', 'b'='bb') as " +
+      "select int_col, string_col from functional.alltypes",
+      "default",
+      "CREATE VIEW test_properties TBLPROPERTIES ('a'='aa', 'b'='bb') AS " +
+      "SELECT int_col, string_col FROM functional.alltypes");
   }
 
   @Test
@@ -534,6 +540,12 @@ public class ToSqlTest extends FrontendTestBase {
         "default", "ALTER VIEW functional.alltypes_view(cnt) AS "+
         "SELECT count(DISTINCT x.int_col) FROM functional.alltypessmall x " +
         "INNER JOIN functional.alltypessmall y ON (x.id = y.id) GROUP BY x.bigint_col");
+    testToSql("alter view functional.alltypes_view set tblproperties ('a'='b')",
+        "functional",
+        "ALTER VIEW functional.alltypes_view SET TBLPROPERTIES ('a'='b')");
+    testToSql("alter view functional.alltypes_view unset tblproperties ('a', 'c')",
+        "functional",
+        "ALTER VIEW functional.alltypes_view UNSET TBLPROPERTIES ('a', 'c')");
   }
 
   @Test
diff --git a/testdata/workloads/functional-query/queries/QueryTest/views-ddl.test b/testdata/workloads/functional-query/queries/QueryTest/views-ddl.test
index a8ce3d5f6..bfcdbe27a 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/views-ddl.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/views-ddl.test
@@ -48,6 +48,14 @@ select * from $DATABASE.complex_view
 'View has been created.'
 ====
 ---- QUERY
+# Create a view with tblproperties
+create view $DATABASE.view_properties
+tblproperties ('a'='aa', 'b'='bb') as
+select * from functional.alltypes
+---- RESULTS
+'View has been created.'
+====
+---- QUERY
 # Test that the views are displayed by 'show tables'
 show tables in $DATABASE
 ---- RESULTS
@@ -55,6 +63,7 @@ show tables in $DATABASE
 'parquet_view'
 'simple_view'
 'simple_view_sub'
+'view_properties'
 'view_view'
 ====
 ---- QUERY
@@ -122,6 +131,25 @@ describe $DATABASE.view_view
 string,string,string
 ====
 ---- QUERY
+describe $DATABASE.view_properties
+---- RESULTS
+'id','int',''
+'bool_col','boolean',''
+'tinyint_col','tinyint',''
+'smallint_col','smallint',''
+'int_col','int',''
+'bigint_col','bigint',''
+'float_col','float',''
+'double_col','double',''
+'date_string_col','string',''
+'string_col','string',''
+'timestamp_col','timestamp',''
+'year','int',''
+'month','int',''
+---- TYPES
+string,string,string
+====
+---- QUERY
 # Test that the views can be queried.
 select count(*) from $DATABASE.simple_view
 ---- RESULTS
@@ -158,6 +186,13 @@ select count(*) from $DATABASE.view_view
 bigint
 ====
 ---- QUERY
+select count(*) from $DATABASE.view_properties
+---- RESULTS
+7300
+---- TYPES
+bigint
+====
+---- QUERY
 # Test dropping a view
 drop view $DATABASE.simple_view_sub
 ---- RESULTS
@@ -170,6 +205,7 @@ show tables in $DATABASE
 'complex_view'
 'parquet_view'
 'simple_view'
+'view_properties'
 'view_view'
 ====
 ---- QUERY
@@ -205,6 +241,7 @@ show tables in $DATABASE
 'complex_view'
 'parquet_view'
 'simple_view'
+'view_properties'
 'view_view'
 ====
 ---- QUERY
@@ -220,6 +257,37 @@ alter view $DATABASE.parquet_view rename to $DATABASE.new_parquet_view
 'Renaming was successful.'
 ====
 ---- QUERY
+# Test set tblproperties for view
+alter view $DATABASE.view_properties set tblproperties ('c'='cc', 'd'='dd')
+---- RESULTS
+'Updated view.'
+====
+---- QUERY
+# Test unset tblproperties for view
+alter view $DATABASE.view_properties unset tblproperties ('c')
+---- RESULTS
+'Updated view.'
+====
+---- QUERY
+# Test DESCRIBE FORMATTED for view with tblproperties
+alter view $DATABASE.view_properties set tblproperties ('e'='ee', 'f'='ff');
+alter view $DATABASE.view_properties unset tblproperties ('e');
+DESCRIBE FORMATTED $DATABASE.view_properties;
+---- RESULTS: VERIFY_IS_NOT_IN
+'','a                  ','aa                  '
+'','b                  ','bb                  '
+'','d                  ','dd                  '
+'','f                  ','ff                  '
+---- TYPES
+string, string, string
+====
+---- QUERY
+# Test dropping a view with tblproperties
+drop view $DATABASE.view_properties
+---- RESULTS
+'View has been dropped.'
+====
+---- QUERY
 # Test that the view was renamed
 show tables in $DATABASE
 ---- RESULTS
@@ -384,4 +452,4 @@ select * from $DATABASE.date_view
 31,9999-12-31,9999-12-31
 ---- TYPES
 int,date,date
-====
+====
\ No newline at end of file