You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by mi...@apache.org on 2018/07/16 15:53:58 UTC

[5/5] impala git commit: Revert "Revert "IMPALA-6988: Implement ALTER TABLE/VIEW SET OWNER""

Revert "Revert "IMPALA-6988: Implement ALTER TABLE/VIEW SET OWNER""

This reverts commit c01efd09679faaacfd5488fc7f4c1526a1af2f35.

Change-Id: I47ed9c6bb983120afdafadea8b65a9239cbf6d0f
Reviewed-on: http://gerrit.cloudera.org:8080/10947
Reviewed-by: Fredy Wijaya <fw...@cloudera.com>
Tested-by: Impala Public Jenkins <im...@cloudera.com>


Project: http://git-wip-us.apache.org/repos/asf/impala/repo
Commit: http://git-wip-us.apache.org/repos/asf/impala/commit/df78eaec
Tree: http://git-wip-us.apache.org/repos/asf/impala/tree/df78eaec
Diff: http://git-wip-us.apache.org/repos/asf/impala/diff/df78eaec

Branch: refs/heads/master
Commit: df78eaec0b42d7fa5d148238f40fa57fcfcb25b5
Parents: a203733
Author: Fredy Wijaya <fw...@cloudera.com>
Authored: Sun Jul 15 10:45:06 2018 -0700
Committer: Impala Public Jenkins <im...@cloudera.com>
Committed: Sun Jul 15 21:50:23 2018 +0000

----------------------------------------------------------------------
 common/thrift/JniCatalog.thrift                 | 13 ++++
 fe/src/main/cup/sql-parser.cup                  | 36 +++++++++-
 .../analysis/AlterTableOrViewSetOwnerStmt.java  | 72 ++++++++++++++++++++
 .../impala/analysis/AlterTableSetOwnerStmt.java | 37 ++++++++++
 .../apache/impala/analysis/AlterTableStmt.java  |  2 +-
 .../impala/analysis/AlterViewSetOwnerStmt.java  | 37 ++++++++++
 .../impala/service/CatalogOpExecutor.java       | 16 ++++-
 .../apache/impala/analysis/AnalyzeDDLTest.java  | 38 +++++++++++
 .../impala/analysis/AuthorizationStmtTest.java  | 23 ++++++-
 .../org/apache/impala/analysis/ParserTest.java  | 26 +++++++
 tests/metadata/test_ddl.py                      | 22 ++++++
 tests/metadata/test_ddl_base.py                 | 19 ++++++
 12 files changed, 335 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/df78eaec/common/thrift/JniCatalog.thrift
----------------------------------------------------------------------
diff --git a/common/thrift/JniCatalog.thrift b/common/thrift/JniCatalog.thrift
index 1876138..4a8298b 100644
--- a/common/thrift/JniCatalog.thrift
+++ b/common/thrift/JniCatalog.thrift
@@ -104,6 +104,7 @@ enum TAlterTableType {
   SET_CACHED,
   RECOVER_PARTITIONS,
   SET_ROW_FORMAT,
+  SET_OWNER
 }
 
 // Parameters of CREATE DATABASE commands
@@ -317,6 +318,15 @@ struct TAlterTableSetLocationParams {
   2: optional list<CatalogObjects.TPartitionKeyValue> partition_spec
 }
 
+// Parameters for ALTER TABLE/VIEW SET OWNER commands.
+struct TAlterTableOrViewSetOwnerParams {
+  // The owner type.
+  1: required TOwnerType owner_type
+
+  // The owner name.
+  2: required string owner_name
+}
+
 // Parameters for updating the table and/or column statistics
 // of a table. Used by ALTER TABLE SET COLUMN STATS, and internally by
 // a COMPUTE STATS command.
@@ -397,6 +407,9 @@ struct TAlterTableParams {
 
   // Parameters for ALTER TABLE SET ROW FORMAT
   15: optional TAlterTableSetRowFormatParams set_row_format_params
+
+  // Parameters for ALTER TABLE/VIEW SET OWNER
+  16: optional TAlterTableOrViewSetOwnerParams set_owner_params
 }
 
 // Parameters of CREATE TABLE LIKE commands

http://git-wip-us.apache.org/repos/asf/impala/blob/df78eaec/fe/src/main/cup/sql-parser.cup
----------------------------------------------------------------------
diff --git a/fe/src/main/cup/sql-parser.cup b/fe/src/main/cup/sql-parser.cup
index bd00ad4..94b69e4 100644
--- a/fe/src/main/cup/sql-parser.cup
+++ b/fe/src/main/cup/sql-parser.cup
@@ -1058,6 +1058,9 @@ alter_db_stmt ::=
   :}
   ;
 
+// In some places, the opt_partition_set is used to avoid conflicts even though
+// a partition clause does not make sense for this stmt. If a partition
+// is given, manually throw a parse error.
 alter_tbl_stmt ::=
   KW_ALTER KW_TABLE table_name:table replace_existing_cols_val:replace KW_COLUMNS
   LPAREN column_def_list:col_defs RPAREN
@@ -1107,9 +1110,7 @@ alter_tbl_stmt ::=
   | KW_ALTER KW_TABLE table_name:table opt_partition_set:partition KW_SET
     KW_COLUMN KW_STATS ident_or_default:col LPAREN properties_map:map RPAREN
   {:
-    // The opt_partition_set is used to avoid conflicts even though
-    // a partition clause does not make sense for this stmt. If a partition
-    // is given, manually throw a parse error.
+    // See above for special partition clause handling.
     if (partition != null) parser.parseError("set", SqlParserSymbols.KW_SET);
     RESULT = new AlterTableSetColumnStats(table, col, map);
   :}
@@ -1134,6 +1135,23 @@ alter_tbl_stmt ::=
   | KW_ALTER KW_TABLE table_name:table KW_ALTER opt_kw_column ident_or_default:col_name
     KW_DROP KW_DEFAULT
   {: RESULT = AlterTableAlterColStmt.createDropDefaultStmt(table, col_name); :}
+  | KW_ALTER KW_TABLE table_name:table opt_partition_set:partitions KW_SET IDENT:owner_id
+    IDENT:user_id ident_or_default:user
+  {:
+    // See above for special partition clause handling.
+    if (partitions != null) parser.parseError("set", SqlParserSymbols.KW_SET);
+    parser.checkIdentKeyword("OWNER", owner_id);
+    parser.checkIdentKeyword("USER", user_id);
+    RESULT = new AlterTableSetOwnerStmt(table, new Owner(TOwnerType.USER, user));
+  :}
+  | KW_ALTER KW_TABLE table_name:table opt_partition_set:partitions KW_SET IDENT:owner_id
+    KW_ROLE ident_or_default:role
+  {:
+    // See above for special partition clause handling.
+    if (partitions != null) parser.parseError("set", SqlParserSymbols.KW_SET);
+    parser.checkIdentKeyword("OWNER", owner_id);
+    RESULT = new AlterTableSetOwnerStmt(table, new Owner(TOwnerType.ROLE, role));
+  :}
   ;
 
 table_property_type ::=
@@ -1877,6 +1895,18 @@ alter_view_stmt ::=
   {: RESULT = new AlterViewStmt(table, col_defs, view_def); :}
   | KW_ALTER KW_VIEW table_name:before_table KW_RENAME KW_TO table_name:new_table
   {: RESULT = new AlterTableOrViewRenameStmt(before_table, new_table, false); :}
+  | KW_ALTER KW_VIEW table_name:table KW_SET IDENT:owner_id IDENT:user_id
+    ident_or_default:user
+  {:
+    parser.checkIdentKeyword("OWNER", owner_id);
+    parser.checkIdentKeyword("USER", user_id);
+    RESULT = new AlterViewSetOwnerStmt(table, new Owner(TOwnerType.USER, user));
+  :}
+  | KW_ALTER KW_VIEW table_name:table KW_SET IDENT:owner_id KW_ROLE ident_or_default:role
+  {:
+    parser.checkIdentKeyword("OWNER", owner_id);
+    RESULT = new AlterViewSetOwnerStmt(table, new Owner(TOwnerType.ROLE, role));
+  :}
   ;
 
 cascade_val ::=

http://git-wip-us.apache.org/repos/asf/impala/blob/df78eaec/fe/src/main/java/org/apache/impala/analysis/AlterTableOrViewSetOwnerStmt.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/analysis/AlterTableOrViewSetOwnerStmt.java b/fe/src/main/java/org/apache/impala/analysis/AlterTableOrViewSetOwnerStmt.java
new file mode 100644
index 0000000..c508413
--- /dev/null
+++ b/fe/src/main/java/org/apache/impala/analysis/AlterTableOrViewSetOwnerStmt.java
@@ -0,0 +1,72 @@
+// 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.impala.authorization.Privilege;
+import org.apache.impala.common.AnalysisException;
+import org.apache.impala.thrift.TAlterTableOrViewSetOwnerParams;
+import org.apache.impala.thrift.TAlterTableParams;
+import org.apache.impala.thrift.TAlterTableType;
+import org.apache.impala.util.MetaStoreUtil;
+
+/**
+ * A base class for ALTER TABLE/VIEW SET OWNER.
+ */
+public abstract class AlterTableOrViewSetOwnerStmt extends AlterTableStmt {
+  protected final Owner owner_;
+
+  public AlterTableOrViewSetOwnerStmt(TableName tableName, Owner owner) {
+    super(tableName);
+    Preconditions.checkNotNull(owner);
+    owner_ = owner;
+  }
+
+  @Override
+  public void analyze(Analyzer analyzer) throws AnalysisException {
+    String ownerName = owner_.getOwnerName();
+    if (ownerName.length() > MetaStoreUtil.MAX_OWNER_LENGTH) {
+      throw new AnalysisException(String.format("Owner name exceeds maximum length of " +
+          "%d characters. The given owner name has %d characters.",
+          MetaStoreUtil.MAX_OWNER_LENGTH, ownerName.length()));
+    }
+    tableName_ = analyzer.getFqTableName(tableName_);
+    TableRef tableRef = new TableRef(tableName_.toPath(), null, Privilege.ALTER);
+    tableRef = analyzer.resolveTableRef(tableRef);
+    Preconditions.checkNotNull(tableRef);
+    tableRef.analyze(analyzer);
+    validateType(tableRef);
+  }
+
+  /**
+   * Validates the type of the given TableRef.
+   */
+  protected abstract void validateType(TableRef tableRef) throws AnalysisException;
+
+  @Override
+  public TAlterTableParams toThrift() {
+    TAlterTableParams params = new TAlterTableParams();
+    params.setTable_name(tableName_.toThrift());
+    TAlterTableOrViewSetOwnerParams ownerParams = new TAlterTableOrViewSetOwnerParams();
+    ownerParams.setOwner_type(owner_.getOwnerType());
+    ownerParams.setOwner_name(owner_.getOwnerName());
+    params.setAlter_type(TAlterTableType.SET_OWNER);
+    params.setSet_owner_params(ownerParams);
+    return params;
+  }
+}

http://git-wip-us.apache.org/repos/asf/impala/blob/df78eaec/fe/src/main/java/org/apache/impala/analysis/AlterTableSetOwnerStmt.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/analysis/AlterTableSetOwnerStmt.java b/fe/src/main/java/org/apache/impala/analysis/AlterTableSetOwnerStmt.java
new file mode 100644
index 0000000..5932aff
--- /dev/null
+++ b/fe/src/main/java/org/apache/impala/analysis/AlterTableSetOwnerStmt.java
@@ -0,0 +1,37 @@
+// 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 org.apache.impala.common.AnalysisException;
+
+/**
+ * Represents an ALTER TABLE tbl SET OWNER [USER|ROLE] owner statement.
+ */
+public class AlterTableSetOwnerStmt extends AlterTableOrViewSetOwnerStmt {
+  public AlterTableSetOwnerStmt(TableName tableName, Owner owner) {
+    super(tableName, owner);
+  }
+
+  @Override
+  protected void validateType(TableRef tableRef) throws AnalysisException {
+    if (tableRef instanceof InlineViewRef) {
+      throw new AnalysisException(String.format(
+          "ALTER TABLE not allowed on a view: %s", tableName_));
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/impala/blob/df78eaec/fe/src/main/java/org/apache/impala/analysis/AlterTableStmt.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/analysis/AlterTableStmt.java b/fe/src/main/java/org/apache/impala/analysis/AlterTableStmt.java
index a173975..0089f0d 100644
--- a/fe/src/main/java/org/apache/impala/analysis/AlterTableStmt.java
+++ b/fe/src/main/java/org/apache/impala/analysis/AlterTableStmt.java
@@ -33,7 +33,7 @@ import com.google.common.base.Preconditions;
  * Base class for all ALTER TABLE statements.
  */
 public abstract class AlterTableStmt extends StatementBase {
-  protected final TableName tableName_;
+  protected TableName tableName_;
 
   // Set during analysis.
   protected FeTable table_;

http://git-wip-us.apache.org/repos/asf/impala/blob/df78eaec/fe/src/main/java/org/apache/impala/analysis/AlterViewSetOwnerStmt.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/analysis/AlterViewSetOwnerStmt.java b/fe/src/main/java/org/apache/impala/analysis/AlterViewSetOwnerStmt.java
new file mode 100644
index 0000000..4e1fc52
--- /dev/null
+++ b/fe/src/main/java/org/apache/impala/analysis/AlterViewSetOwnerStmt.java
@@ -0,0 +1,37 @@
+// 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 org.apache.impala.common.AnalysisException;
+
+/**
+ * Represents an ALTER VIEW v SET OWNER [USER|ROLE] owner statement.
+ */
+public class AlterViewSetOwnerStmt extends AlterTableOrViewSetOwnerStmt {
+  public AlterViewSetOwnerStmt(TableName tableName, Owner owner) {
+    super(tableName, owner);
+  }
+
+  @Override
+  protected void validateType(TableRef tableRef) throws AnalysisException {
+    if (!(tableRef instanceof InlineViewRef)) {
+      throw new AnalysisException(String.format(
+          "ALTER VIEW not allowed on a table: %s", tableName_));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/impala/blob/df78eaec/fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java b/fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java
index 554d2c3..39cc108 100644
--- a/fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java
+++ b/fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java
@@ -100,6 +100,7 @@ import org.apache.impala.thrift.TAlterTableAddReplaceColsParams;
 import org.apache.impala.thrift.TAlterTableAlterColParams;
 import org.apache.impala.thrift.TAlterTableDropColParams;
 import org.apache.impala.thrift.TAlterTableDropPartitionParams;
+import org.apache.impala.thrift.TAlterTableOrViewSetOwnerParams;
 import org.apache.impala.thrift.TAlterTableParams;
 import org.apache.impala.thrift.TAlterTableSetCachedParams;
 import org.apache.impala.thrift.TAlterTableSetFileFormatParams;
@@ -557,6 +558,11 @@ public class CatalogOpExecutor {
           alterTableRecoverPartitions(tbl);
           addSummary(response, "Partitions have been recovered.");
           break;
+        case SET_OWNER:
+          Preconditions.checkState(params.isSetSet_owner_params());
+          alterTableOrViewSetOwner(tbl, params.getSet_owner_params());
+          addSummary(response, "Updated table/view.");
+          break;
         default:
           throw new UnsupportedOperationException(
               "Unknown ALTER TABLE operation type: " + params.getAlter_type());
@@ -2799,6 +2805,14 @@ public class CatalogOpExecutor {
     }
   }
 
+  private void alterTableOrViewSetOwner(Table tbl, TAlterTableOrViewSetOwnerParams params)
+      throws ImpalaRuntimeException {
+    org.apache.hadoop.hive.metastore.api.Table msTbl = tbl.getMetaStoreTable().deepCopy();
+    msTbl.setOwner(params.owner_name);
+    msTbl.setOwnerType(PrincipalType.valueOf(params.owner_type.name()));
+    applyAlterTable(msTbl, true);
+  }
+
   /**
    * Create a new HMS Partition.
    */
@@ -3564,7 +3578,7 @@ public class CatalogOpExecutor {
       }
     }
     addDbToCatalogUpdate(db, response.result);
-    addSummary(response, "Updated database");
+    addSummary(response, "Updated database.");
   }
 
   private void addDbToCatalogUpdate(Db db, TCatalogUpdateResult result) {

http://git-wip-us.apache.org/repos/asf/impala/blob/df78eaec/fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java
----------------------------------------------------------------------
diff --git a/fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java b/fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java
index c565b45..46b3787 100644
--- a/fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java
+++ b/fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java
@@ -4059,6 +4059,44 @@ public class AnalyzeDDLTest extends FrontendTestBase {
     }
   }
 
+  @Test
+  public void TestAlterTableSetOwner() {
+    String[] ownerTypes = new String[]{"user", "role"};
+    for (String ownerType : ownerTypes) {
+      AnalyzesOk(String.format("alter table functional.alltypes set owner %s foo",
+          ownerType));
+      AnalysisError(String.format("alter table nodb.alltypes set owner %s foo",
+          ownerType), "Could not resolve table reference: 'nodb.alltypes'");
+      AnalysisError(String.format("alter table functional.notbl set owner %s foo",
+          ownerType), "Could not resolve table reference: 'functional.notbl'");
+      AnalysisError(String.format("alter table functional.alltypes set owner %s %s",
+          ownerType, buildLongOwnerName()), "Owner name exceeds maximum length of 128 " +
+          "characters. The given owner name has 133 characters.");
+      AnalysisError(String.format("alter table functional.alltypes_view " +
+          "set owner %s foo", ownerType), "ALTER TABLE not allowed on a view: " +
+          "functional.alltypes_view");
+    }
+  }
+
+  @Test
+  public void TestAlterViewSetOwner() {
+    String[] ownerTypes = new String[]{"user", "role"};
+    for (String ownerType : ownerTypes) {
+      AnalyzesOk(String.format("alter view functional.alltypes_view set owner %s foo",
+          ownerType));
+      AnalysisError(String.format("alter view nodb.alltypes set owner %s foo",
+          ownerType), "Could not resolve table reference: 'nodb.alltypes'");
+      AnalysisError(String.format("alter view functional.notbl set owner %s foo",
+          ownerType), "Could not resolve table reference: 'functional.notbl'");
+      AnalysisError(String.format("alter view functional.alltypes_view set owner %s %s",
+          ownerType, buildLongOwnerName()), "Owner name exceeds maximum length of 128 " +
+          "characters. The given owner name has 133 characters.");
+      AnalysisError(String.format("alter view functional.alltypes " +
+          "set owner %s foo", ownerType), "ALTER VIEW not allowed on a table: " +
+          "functional.alltypes");
+    }
+  }
+
   private static String buildLongOwnerName() {
     StringBuilder comment = new StringBuilder();
     for (int i = 0; i < MetaStoreUtil.MAX_OWNER_LENGTH + 5; i++) {

http://git-wip-us.apache.org/repos/asf/impala/blob/df78eaec/fe/src/test/java/org/apache/impala/analysis/AuthorizationStmtTest.java
----------------------------------------------------------------------
diff --git a/fe/src/test/java/org/apache/impala/analysis/AuthorizationStmtTest.java b/fe/src/test/java/org/apache/impala/analysis/AuthorizationStmtTest.java
index 524aff3..52e0952 100644
--- a/fe/src/test/java/org/apache/impala/analysis/AuthorizationStmtTest.java
+++ b/fe/src/test/java/org/apache/impala/analysis/AuthorizationStmtTest.java
@@ -1652,7 +1652,9 @@ public class AuthorizationStmtTest extends FrontendTestBase {
             "delimited fields terminated by ' '"),
         authorize("alter table functional.alltypes add partition(year=1, month=1)"),
         authorize("alter table functional.alltypes drop partition(" +
-            "year=2009, month=1)")}) {
+            "year=2009, month=1)"),
+        authorize("alter table functional.alltypes set owner user foo_owner"),
+        authorize("alter table functional.alltypes set owner role foo_owner")}) {
       test.ok(onServer(TPrivilegeLevel.ALL))
           .ok(onServer(TPrivilegeLevel.ALTER))
           .ok(onDatabase("functional", TPrivilegeLevel.ALL))
@@ -1825,6 +1827,25 @@ public class AuthorizationStmtTest extends FrontendTestBase {
         .error(alterError("functional.alltypes_view"), onTable("functional",
             "alltypes_view", allExcept(TPrivilegeLevel.ALL, TPrivilegeLevel.ALTER)));
 
+    // Alter view set owner.
+    for (AuthzTest test: new AuthzTest[]{
+        authorize("alter view functional.alltypes_view set owner user foo_owner"),
+        authorize("alter view functional.alltypes_view set owner role foo_owner")}) {
+      test.ok(onServer(TPrivilegeLevel.ALL))
+          .ok(onServer(TPrivilegeLevel.ALTER))
+          .ok(onDatabase("functional", TPrivilegeLevel.ALL))
+          .ok(onDatabase("functional", TPrivilegeLevel.ALTER))
+          .ok(onTable("functional", "alltypes_view", TPrivilegeLevel.ALL))
+          .ok(onTable("functional", "alltypes_view", TPrivilegeLevel.ALTER))
+          .error(alterError("functional.alltypes_view"))
+          .error(alterError("functional.alltypes_view"), onServer(allExcept(
+              TPrivilegeLevel.ALL, TPrivilegeLevel.ALTER)))
+          .error(alterError("functional.alltypes_view"), onDatabase("functional", allExcept(
+              TPrivilegeLevel.ALL, TPrivilegeLevel.ALTER)))
+          .error(alterError("functional.alltypes_view"), onTable("functional",
+              "alltypes_view", allExcept(TPrivilegeLevel.ALL, TPrivilegeLevel.ALTER)));
+    }
+
     // Database does not exist.
     authorize("alter view nodb.alltypes_view as select 1")
         .error(alterError("nodb"))

http://git-wip-us.apache.org/repos/asf/impala/blob/df78eaec/fe/src/test/java/org/apache/impala/analysis/ParserTest.java
----------------------------------------------------------------------
diff --git a/fe/src/test/java/org/apache/impala/analysis/ParserTest.java b/fe/src/test/java/org/apache/impala/analysis/ParserTest.java
index 6059b8b..a86f467 100644
--- a/fe/src/test/java/org/apache/impala/analysis/ParserTest.java
+++ b/fe/src/test/java/org/apache/impala/analysis/ParserTest.java
@@ -3809,4 +3809,30 @@ public class ParserTest extends FrontendTestBase {
     ParserError("ALTER DATABASE SET OWNER ROLE foo");
     ParserError("ALTER DATABASE SET OWNER");
   }
+
+  @Test
+  public void TestAlterTableOrViewSetOwner() {
+    for (String type : new String[]{"TABLE", "VIEW"}) {
+      for (String valid : new String[]{"foo", "user", "owner"}) {
+        ParsesOk(String.format("ALTER %s %s SET OWNER USER %s", type, valid, valid));
+        ParsesOk(String.format("ALTER %s %s SET OWNER ROLE %s", type, valid, valid));
+      }
+
+      for (String invalid : new String[]{"'foo'", "''", "NULL"}) {
+        ParserError(String.format("ALTER %s %s SET OWNER ROLE %s", type, invalid, invalid));
+        ParserError(String.format("ALTER %s %s SET OWNER USER %s", type, invalid, invalid));
+      }
+
+      ParserError(String.format("ALTER %s tbl PARTITION(i=1) SET OWNER ROLE foo", type));
+      ParserError(String.format("ALTER %s tbl SET ABC USER foo", type));
+      ParserError(String.format("ALTER %s tbl SET ABC ROLE foo", type));
+      ParserError(String.format("ALTER %s tbl SET OWNER ABC foo", type));
+      ParserError(String.format("ALTER %s tbl SET OWNER USER", type));
+      ParserError(String.format("ALTER %s SET OWNER foo", type));
+      ParserError(String.format("ALTER %s SET OWNER USER foo", type));
+      ParserError(String.format("ALTER %s tbl SET OWNER ROLE", type));
+      ParserError(String.format("ALTER %s SET OWNER ROLE foo", type));
+      ParserError(String.format("ALTER %s SET OWNER", type));
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/impala/blob/df78eaec/tests/metadata/test_ddl.py
----------------------------------------------------------------------
diff --git a/tests/metadata/test_ddl.py b/tests/metadata/test_ddl.py
index 7d2b714..ff6a566 100644
--- a/tests/metadata/test_ddl.py
+++ b/tests/metadata/test_ddl.py
@@ -227,6 +227,28 @@ class TestDdlStatements(TestDdlBase):
     assert len(properties) == 1
     assert {'foo_role': 'ROLE'} == properties
 
+  def test_alter_table_set_owner(self, vector, unique_database):
+    table_name = "{0}.test_owner_tbl".format(unique_database)
+    self.client.execute("create table {0}(i int)".format(table_name))
+    self.client.execute("alter table {0} set owner user foo_user".format(table_name))
+    owner = self._get_table_or_view_owner(table_name)
+    assert ('foo_user', 'USER') == owner
+
+    self.client.execute("alter table {0} set owner role foo_role".format(table_name))
+    owner = self._get_table_or_view_owner(table_name)
+    assert ('foo_role', 'ROLE') == owner
+
+  def test_alter_view_set_owner(self, vector, unique_database):
+    view_name = "{0}.test_owner_tbl".format(unique_database)
+    self.client.execute("create view {0} as select 1".format(view_name))
+    self.client.execute("alter view {0} set owner user foo_user".format(view_name))
+    owner = self._get_table_or_view_owner(view_name)
+    assert ('foo_user', 'USER') == owner
+
+    self.client.execute("alter view {0} set owner role foo_role".format(view_name))
+    owner = self._get_table_or_view_owner(view_name)
+    assert ('foo_role', 'ROLE') == owner
+
   # There is a query in QueryTest/create-table that references nested types, which is not
   # supported if old joins and aggs are enabled. Since we do not get any meaningful
   # additional coverage by running a DDL test under the old aggs and joins, it can be

http://git-wip-us.apache.org/repos/asf/impala/blob/df78eaec/tests/metadata/test_ddl_base.py
----------------------------------------------------------------------
diff --git a/tests/metadata/test_ddl_base.py b/tests/metadata/test_ddl_base.py
index bc74e6e..a27aa1c 100644
--- a/tests/metadata/test_ddl_base.py
+++ b/tests/metadata/test_ddl_base.py
@@ -93,6 +93,18 @@ class TestDdlBase(ImpalaTestSuite):
         properties[row[1].rstrip()] = row[2].rstrip()
     return properties
 
+  def _get_property(self, property_name, name, is_db=False):
+    """Extracts a db/table property value from the output of DESCRIBE FORMATTED."""
+    result = self.client.execute("describe {0} formatted {1}".format(
+      "database" if is_db else "", name))
+    for row in result.data:
+      if property_name in row:
+        row = row.split('\t')
+        if row[1] == 'NULL':
+          break
+        return row[1].rstrip()
+    return None
+
   def _get_db_comment(self, db_name):
     """Extracts the DB comment from the output of DESCRIBE DATABASE"""
     result = self.client.execute("describe database {0}".format(db_name))
@@ -110,3 +122,10 @@ class TestDdlBase(ImpalaTestSuite):
       if len(cols) == 3:
         comments[cols[0].rstrip()] = cols[2].rstrip()
     return comments.get(col_name)
+
+
+  def _get_table_or_view_owner(self, table_name):
+    """Returns a tuple(owner, owner_type) for a given table name"""
+    owner_name = self._get_property("Owner:", table_name)
+    owner_type = self._get_property("OwnerType:", table_name)
+    return (owner_name, owner_type)