You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by sp...@apache.org on 2018/05/11 20:27:50 UTC

hive git commit: HIVE-19374: Parse and process ALTER TABLE SET OWNER command syntax (Sergio Pena, reviewed by Vihang Karajgaonkar)

Repository: hive
Updated Branches:
  refs/heads/master c7af8a516 -> 615e71e79


HIVE-19374: Parse and process ALTER TABLE SET OWNER command syntax (Sergio Pena, reviewed by Vihang Karajgaonkar)


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

Branch: refs/heads/master
Commit: 615e71e798b299ff58ef22a1a8f91073b0cefab5
Parents: c7af8a5
Author: Sergio Pena <se...@cloudera.com>
Authored: Wed May 9 22:37:09 2018 -0700
Committer: Sergio Pena <se...@cloudera.com>
Committed: Fri May 11 13:27:10 2018 -0700

----------------------------------------------------------------------
 .../ql/metadata/TestAlterTableMetadata.java     | 72 ++++++++++++++++++++
 .../org/apache/hadoop/hive/ql/exec/DDLTask.java |  5 ++
 .../apache/hadoop/hive/ql/metadata/Table.java   | 15 ++++
 .../formatting/JsonMetaDataFormatter.java       |  1 +
 .../formatting/MetaDataFormatUtils.java         |  1 +
 .../hive/ql/parse/DDLSemanticAnalyzer.java      | 17 +++++
 .../apache/hadoop/hive/ql/parse/HiveParser.g    |  8 +++
 .../hive/ql/parse/SemanticAnalyzerFactory.java  |  2 +
 .../hadoop/hive/ql/plan/AlterTableDesc.java     | 21 +++++-
 .../hadoop/hive/ql/plan/HiveOperation.java      |  1 +
 .../authorization/plugin/HiveOperationType.java |  1 +
 .../plugin/sqlstd/Operation2Privilege.java      |  2 +
 12 files changed, 145 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/615e71e7/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/metadata/TestAlterTableMetadata.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/metadata/TestAlterTableMetadata.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/metadata/TestAlterTableMetadata.java
new file mode 100644
index 0000000..940a1d3
--- /dev/null
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/metadata/TestAlterTableMetadata.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.hadoop.hive.ql.metadata;
+
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.api.PrincipalType;
+import org.apache.hadoop.hive.ql.DriverFactory;
+import org.apache.hadoop.hive.ql.IDriver;
+import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse;
+import org.apache.hadoop.hive.ql.session.SessionState;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestAlterTableMetadata {
+  @Test
+  public void testAlterTableOwner() throws HiveException {
+    /*
+     * This test verifies that the ALTER TABLE ... SET OWNER command will change the
+     * owner metadata of the table in HMS.
+     */
+
+    HiveConf conf = new HiveConf(this.getClass());
+    conf.set(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY.varname, "false");
+    SessionState.start(conf);
+    IDriver driver = DriverFactory.newDriver(conf);
+    CommandProcessorResponse resp;
+    Table table;
+
+    resp = driver.run("create table t1(id int)");
+    assertEquals(0, resp.getResponseCode());
+
+    // Changes the owner to a user and verify the change
+    resp = driver.run("alter table t1 set owner user u1");
+    assertEquals(0, resp.getResponseCode());
+
+    table = Hive.get(conf).getTable("t1");
+    assertEquals(PrincipalType.USER, table.getOwnerType());
+    assertEquals("u1", table.getOwner());
+
+    // Changes the owner to a group and verify the change
+    resp = driver.run("alter table t1 set owner group g1");
+    assertEquals(0, resp.getResponseCode());
+
+    table = Hive.get(conf).getTable("t1");
+    assertEquals(PrincipalType.GROUP, table.getOwnerType());
+    assertEquals("g1", table.getOwner());
+
+    // Changes the owner to a role and verify the change
+    resp = driver.run("alter table t1 set owner role r1");
+    assertEquals(0, resp.getResponseCode());
+
+    table = Hive.get(conf).getTable("t1");
+    assertEquals(PrincipalType.ROLE, table.getOwnerType());
+    assertEquals("r1", table.getOwner());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/615e71e7/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
index abde9f7..63fe8ad 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
@@ -4330,6 +4330,11 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
       }
 
       tbl.setStoredAsSubDirectories(alterTbl.isStoredAsSubDirectories());
+    } else if (alterTbl.getOp() == AlterTableDesc.AlterTableTypes.OWNER) {
+      if (alterTbl.getOwnerPrincipal() != null) {
+        tbl.setOwner(alterTbl.getOwnerPrincipal().getName());
+        tbl.setOwnerType(alterTbl.getOwnerPrincipal().getType());
+      }
     } else if (alterTbl.getOp() == AlterTableDesc.AlterTableTypes.ALTERSKEWEDLOCATION) {
       // process location one-by-one
       Map<List<String>,String> locMaps = alterTbl.getSkewedLocations();

http://git-wip-us.apache.org/repos/asf/hive/blob/615e71e7/ql/src/java/org/apache/hadoop/hive/ql/metadata/Table.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/Table.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/Table.java
index 879b422..ba16f84 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/Table.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/Table.java
@@ -46,6 +46,7 @@ import org.apache.hadoop.hive.metastore.api.CreationMetadata;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.api.Order;
+import org.apache.hadoop.hive.metastore.api.PrincipalType;
 import org.apache.hadoop.hive.metastore.api.SerDeInfo;
 import org.apache.hadoop.hive.metastore.api.SkewedInfo;
 import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
@@ -744,6 +745,12 @@ public class Table implements Serializable {
   }
 
   /**
+   * @return The owner type of the table.
+   * @see org.apache.hadoop.hive.metastore.api.Table#getOwnerType()
+   */
+  public PrincipalType getOwnerType() { return tTable.getOwnerType(); }
+
+  /**
    * @return The table parameters.
    * @see org.apache.hadoop.hive.metastore.api.Table#getParameters()
    */
@@ -768,6 +775,14 @@ public class Table implements Serializable {
   }
 
   /**
+   * @param ownerType
+   * @see org.apache.hadoop.hive.metastore.api.Table#setOwnerType(org.apache.hadoop.hive.metastore.api.PrincipalType)
+   */
+  public void setOwnerType(PrincipalType ownerType) {
+    tTable.setOwnerType(ownerType);
+  }
+
+  /**
    * @param retention
    * @see org.apache.hadoop.hive.metastore.api.Table#setRetention(int)
    */

http://git-wip-us.apache.org/repos/asf/hive/blob/615e71e7/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/JsonMetaDataFormatter.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/JsonMetaDataFormatter.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/JsonMetaDataFormatter.java
index c21967c..e7b5af6 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/JsonMetaDataFormatter.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/JsonMetaDataFormatter.java
@@ -218,6 +218,7 @@ public class JsonMetaDataFormatter implements MetaDataFormatter {
     MapBuilder builder = MapBuilder.create();
 
     builder.put("tableName", tbl.getTableName());
+    builder.put("ownerType", (tbl.getOwnerType() != null) ? tbl.getOwnerType().name() : "null");
     builder.put("owner", tbl.getOwner());
     builder.put("location", tblLoc);
     builder.put("inputFormat", inputFormattCls);

http://git-wip-us.apache.org/repos/asf/hive/blob/615e71e7/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/MetaDataFormatUtils.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/MetaDataFormatUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/MetaDataFormatUtils.java
index af283e6..44687ef 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/MetaDataFormatUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/MetaDataFormatUtils.java
@@ -425,6 +425,7 @@ public final class MetaDataFormatUtils {
   private static void getTableMetaDataInformation(StringBuilder tableInfo, Table  tbl,
       boolean isOutputPadded) {
     formatOutput("Database:", tbl.getDbName(), tableInfo);
+    formatOutput("OwnerType:", (tbl.getOwnerType() != null) ? tbl.getOwnerType().name() : "null", tableInfo);
     formatOutput("Owner:", tbl.getOwner(), tableInfo);
     formatOutput("CreateTime:", formatDate(tbl.getTTable().getCreateTime()), tableInfo);
     formatOutput("LastAccessTime:", formatDate(tbl.getTTable().getLastAccessTime()), tableInfo);

http://git-wip-us.apache.org/repos/asf/hive/blob/615e71e7/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
index f0b9eda..37e9845 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
@@ -365,6 +365,8 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
           analyzeAlterTableAddConstraint(ast, tableName);
       } else if(ast.getToken().getType() == HiveParser.TOK_ALTERTABLE_UPDATECOLUMNS) {
         analyzeAlterTableUpdateColumns(ast, tableName, partSpec);
+      } else if (ast.getToken().getType() == HiveParser.TOK_ALTERTABLE_OWNER) {
+        analyzeAlterTableOwner(ast, tableName);
       }
       break;
     }
@@ -1928,6 +1930,21 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
     }
   }
 
+  private void analyzeAlterTableOwner(ASTNode ast, String tableName) throws SemanticException {
+    PrincipalDesc ownerPrincipal = AuthorizationParseUtils.getPrincipalDesc((ASTNode) ast.getChild(0));
+
+    if (ownerPrincipal.getType() == null) {
+      throw new SemanticException("Owner type can't be null in alter table set owner command");
+    }
+
+    if (ownerPrincipal.getName() == null) {
+      throw new SemanticException("Owner name can't be null in alter table set owner command");
+    }
+
+    AlterTableDesc alterTblDesc  = new AlterTableDesc(tableName, ownerPrincipal);
+    rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), alterTblDesc), conf));
+  }
+
   private void analyzeAlterTableLocation(ASTNode ast, String tableName,
       HashMap<String, String> partSpec) throws SemanticException {
 

http://git-wip-us.apache.org/repos/asf/hive/blob/615e71e7/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g b/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g
index 3712a53..8726974 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g
@@ -188,6 +188,7 @@ TOK_ALTERTABLE_COMPACT;
 TOK_ALTERTABLE_DROPCONSTRAINT;
 TOK_ALTERTABLE_ADDCONSTRAINT;
 TOK_ALTERTABLE_UPDATECOLUMNS;
+TOK_ALTERTABLE_OWNER;
 TOK_MSCK;
 TOK_SHOWDATABASES;
 TOK_SHOWTABLES;
@@ -1136,6 +1137,7 @@ alterTableStatementSuffix
     | alterStatementSuffixDropConstraint
     | alterStatementSuffixAddConstraint
     | partitionSpec? alterTblPartitionStatementSuffix -> alterTblPartitionStatementSuffix partitionSpec?
+    | alterStatementSuffixSetOwner
     ;
 
 alterTblPartitionStatementSuffix
@@ -1481,6 +1483,12 @@ alterStatementSuffixCompact
     -> ^(TOK_ALTERTABLE_COMPACT $compactType blocking? tableProperties?)
     ;
 
+alterStatementSuffixSetOwner
+@init { pushMsg("alter table set owner", state); }
+@after { popMsg(state); }
+    : KW_SET KW_OWNER principalName
+    -> ^(TOK_ALTERTABLE_OWNER principalName)
+    ;
 
 fileFormat
 @init { pushMsg("file format specification", state); }

http://git-wip-us.apache.org/repos/asf/hive/blob/615e71e7/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzerFactory.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzerFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzerFactory.java
index 70295da..088b5cf 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzerFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzerFactory.java
@@ -94,6 +94,7 @@ public final class SemanticAnalyzerFactory {
     commandType.put(HiveParser.TOK_ALTERVIEW_DROPPROPERTIES, HiveOperation.ALTERVIEW_PROPERTIES);
     commandType.put(HiveParser.TOK_ALTERVIEW_ADDPARTS, HiveOperation.ALTERTABLE_ADDPARTS);
     commandType.put(HiveParser.TOK_ALTERVIEW_DROPPARTS, HiveOperation.ALTERTABLE_DROPPARTS);
+    commandType.put(HiveParser.TOK_ALTERTABLE_OWNER, HiveOperation.ALTERTABLE_OWNER);
     commandType.put(HiveParser.TOK_ALTERVIEW_RENAME, HiveOperation.ALTERVIEW_RENAME);
     commandType.put(HiveParser.TOK_ALTERVIEW, HiveOperation.ALTERVIEW_AS);
     commandType.put(HiveParser.TOK_ALTER_MATERIALIZED_VIEW_REWRITE,
@@ -244,6 +245,7 @@ public final class SemanticAnalyzerFactory {
           case HiveParser.TOK_ALTERTABLE_DROPPROPERTIES:
           case HiveParser.TOK_ALTERTABLE_EXCHANGEPARTITION:
           case HiveParser.TOK_ALTERTABLE_SKEWED:
+          case HiveParser.TOK_ALTERTABLE_OWNER:
           case HiveParser.TOK_ALTERTABLE_DROPCONSTRAINT:
           case HiveParser.TOK_ALTERTABLE_ADDCONSTRAINT:
           case HiveParser.TOK_ALTERTABLE_UPDATECOLUMNS:

http://git-wip-us.apache.org/repos/asf/hive/blob/615e71e7/ql/src/java/org/apache/hadoop/hive/ql/plan/AlterTableDesc.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/AlterTableDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/AlterTableDesc.java
index a767796..0b04c0c 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/AlterTableDesc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/AlterTableDesc.java
@@ -66,7 +66,7 @@ public class AlterTableDesc extends DDLDesc implements Serializable, DDLDesc.DDL
     ALTERSKEWEDLOCATION("alter skew location"), ALTERBUCKETNUM("alter bucket number"),
     ALTERPARTITION("alter partition"), COMPACT("compact"),
     TRUNCATE("truncate"), MERGEFILES("merge files"), DROPCONSTRAINT("drop constraint"), ADDCONSTRAINT("add constraint"),
-    UPDATECOLUMNS("update columns");
+    UPDATECOLUMNS("update columns"), OWNER("set owner");
     ;
 
     private final String name;
@@ -138,6 +138,7 @@ public class AlterTableDesc extends DDLDesc implements Serializable, DDLDesc.DDL
   List<SQLCheckConstraint> checkConstraintsCols;
   ReplicationSpec replicationSpec;
   private Long writeId = null;
+  PrincipalDesc ownerPrincipal;
 
   public AlterTableDesc() {
   }
@@ -367,6 +368,24 @@ public class AlterTableDesc extends DDLDesc implements Serializable, DDLDesc.DDL
     op = AlterTableTypes.ADDCONSTRAINT;
   }
 
+  public AlterTableDesc(String tableName, PrincipalDesc ownerPrincipal) {
+    op  = AlterTableTypes.OWNER;
+    this.oldName = tableName;
+    this.ownerPrincipal = ownerPrincipal;
+  }
+
+  /**
+   * @param ownerPrincipal the owner principal of the table
+   */
+  public void setOwnerPrincipal(PrincipalDesc ownerPrincipal) {
+    this.ownerPrincipal = ownerPrincipal;
+  }
+
+  @Explain(displayName="owner")
+  public PrincipalDesc getOwnerPrincipal() {
+    return this.ownerPrincipal;
+  }
+
   @Explain(displayName = "new columns", explainLevels = { Level.USER, Level.DEFAULT, Level.EXTENDED })
   public List<String> getNewColsString() {
     return Utilities.getFieldSchemaString(getNewCols());

http://git-wip-us.apache.org/repos/asf/hive/blob/615e71e7/ql/src/java/org/apache/hadoop/hive/ql/plan/HiveOperation.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/HiveOperation.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/HiveOperation.java
index cd4c206..ba3330b 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/HiveOperation.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/HiveOperation.java
@@ -51,6 +51,7 @@ public enum HiveOperation {
   ALTERTABLE_ARCHIVE("ALTERTABLE_ARCHIVE", new Privilege[]{Privilege.ALTER_DATA}, null),
   ALTERTABLE_UNARCHIVE("ALTERTABLE_UNARCHIVE", new Privilege[]{Privilege.ALTER_DATA}, null),
   ALTERTABLE_PROPERTIES("ALTERTABLE_PROPERTIES", new Privilege[]{Privilege.ALTER_METADATA}, null),
+  ALTERTABLE_OWNER("ALTERTABLE_OWNER", null, null),
   ALTERTABLE_SERIALIZER("ALTERTABLE_SERIALIZER", new Privilege[]{Privilege.ALTER_METADATA}, null),
   ALTERPARTITION_SERIALIZER("ALTERPARTITION_SERIALIZER", new Privilege[]{Privilege.ALTER_METADATA}, null),
   ALTERTABLE_SERDEPROPERTIES("ALTERTABLE_SERDEPROPERTIES", new Privilege[]{Privilege.ALTER_METADATA}, null),

http://git-wip-us.apache.org/repos/asf/hive/blob/615e71e7/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveOperationType.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveOperationType.java b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveOperationType.java
index be5c062..09c15f0 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveOperationType.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveOperationType.java
@@ -53,6 +53,7 @@ public enum HiveOperationType {
   ALTERTABLE_ARCHIVE,
   ALTERTABLE_UNARCHIVE,
   ALTERTABLE_PROPERTIES,
+  ALTERTABLE_OWNER,
   ALTERTABLE_SERIALIZER,
   ALTERTABLE_PARTCOLTYPE,
   ALTERTABLE_DROPCONSTRAINT,

http://git-wip-us.apache.org/repos/asf/hive/blob/615e71e7/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/Operation2Privilege.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/Operation2Privilege.java b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/Operation2Privilege.java
index 771dfb7..6137983 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/Operation2Privilege.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/Operation2Privilege.java
@@ -196,6 +196,8 @@ public class Operation2Privilege {
 (OWNER_PRIV_AR, OWNER_PRIV_AR));
     op2Priv.put(HiveOperationType.ALTERTABLE_PROPERTIES, PrivRequirement.newIOPrivRequirement
 (OWNER_PRIV_AR, OWNER_PRIV_AR));
+    op2Priv.put(HiveOperationType.ALTERTABLE_OWNER, PrivRequirement.newIOPrivRequirement
+(OWNER_PRIV_AR, OWNER_PRIV_AR));
     op2Priv.put(HiveOperationType.ALTERTABLE_SERIALIZER, PrivRequirement.newIOPrivRequirement
 (OWNER_PRIV_AR, OWNER_PRIV_AR));
     op2Priv.put(HiveOperationType.ALTERTABLE_PARTCOLTYPE, PrivRequirement.newIOPrivRequirement