You are viewing a plain text version of this content. The canonical link for it is here.
Posted to gitbox@hive.apache.org by GitBox <gi...@apache.org> on 2021/06/04 10:06:03 UTC

[GitHub] [hive] szlta opened a new pull request #2351: HIVE-25200: Alter table add columns support for Iceberg tables

szlta opened a new pull request #2351:
URL: https://github.com/apache/hive/pull/2351


   Since Iceberg counts as being a non-native Hive table, addColumn operation needs to be implemented by the help of Hive meta hooks.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] marton-bod commented on a change in pull request #2351: HIVE-25200: Alter table add columns support for Iceberg tables

Posted by GitBox <gi...@apache.org>.
marton-bod commented on a change in pull request #2351:
URL: https://github.com/apache/hive/pull/2351#discussion_r647344554



##########
File path: iceberg/iceberg-catalog/src/main/java/org/apache/iceberg/hive/HiveSchemaUtil.java
##########
@@ -134,6 +137,23 @@ public static Type convert(TypeInfo typeInfo) {
     return HiveSchemaConverter.convert(typeInfo, false);
   }
 
+  /**
+   * Produces the difference of two FieldSchema lists by only taking into account the field name and type.
+   * @param subtrahendCollection List of fields to subtract from

Review comment:
       I think minuend and subtrahend are the other way around in this case




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] marton-bod commented on a change in pull request #2351: HIVE-25200: Alter table add columns support for Iceberg tables

Posted by GitBox <gi...@apache.org>.
marton-bod commented on a change in pull request #2351:
URL: https://github.com/apache/hive/pull/2351#discussion_r647346665



##########
File path: iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergMetaHook.java
##########
@@ -84,6 +88,8 @@
       // Initially we'd like to cache the partition spec in HMS, but not push it down later to Iceberg during alter
       // table commands since by then the HMS info can be stale + Iceberg does not store its partition spec in the props
       InputFormatConfig.PARTITION_SPEC);
+  private static final Set<Enum<?>> SUPPORTED_ALTER_OPS = ImmutableSet.of(

Review comment:
       Maybe we should use EnumSet here instead?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] pvary commented on a change in pull request #2351: HIVE-25200: Alter table add columns support for Iceberg tables

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #2351:
URL: https://github.com/apache/hive/pull/2351#discussion_r645946917



##########
File path: iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerNoScan.java
##########
@@ -783,6 +785,43 @@ public void testDropHiveTableWithoutUnderlyingTable() throws IOException {
     shell.executeStatement("DROP TABLE " + identifier);
   }
 
+  @Test
+  public void testAlterTableAddColumns() throws Exception {
+    Assume.assumeTrue("Iceberg - alter table/add column is only relevant for HiveCatalog",

Review comment:
       We do not support this for other catalogs? 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] szlta commented on a change in pull request #2351: HIVE-25200: Alter table add columns support for Iceberg tables

Posted by GitBox <gi...@apache.org>.
szlta commented on a change in pull request #2351:
URL: https://github.com/apache/hive/pull/2351#discussion_r647344754



##########
File path: iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergMetaHook.java
##########
@@ -310,6 +335,24 @@ public void rollbackAlterTable(org.apache.hadoop.hive.metastore.api.Table hmsTab
     }
   }
 
+  private void setupAlterOperationType(EnvironmentContext context) throws MetaException {
+    if (context != null) {
+      Map<String, String> contextProperties = context.getProperties();
+      if (contextProperties != null) {
+        String stringOpType = contextProperties.get(ALTER_TABLE_OPERATION_TYPE);
+        if (stringOpType != null) {
+          currentAlterTableOp = AlterTableType.valueOf(stringOpType);
+          if (SUPPORTED_ALTER_OPS.stream().noneMatch(op -> op.equals(currentAlterTableOp))) {
+            throw new MetaException(
+                "Unsupported ALTER TABLE operation type for Iceberg tables, must be: " + allowedAlterTypes.toString());
+          }
+        }
+        return;

Review comment:
       Yeah I found that it is valid as tests started to fail after the recent refactor :D E.g. for analyze+compute_stats query there's an alter table invocation, where there's no operation type among the context properties. Our hook should not fail for such cases, but rather act as no-op.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] pvary commented on a change in pull request #2351: HIVE-25200: Alter table add columns support for Iceberg tables

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #2351:
URL: https://github.com/apache/hive/pull/2351#discussion_r645947231



##########
File path: iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerNoScan.java
##########
@@ -783,6 +785,43 @@ public void testDropHiveTableWithoutUnderlyingTable() throws IOException {
     shell.executeStatement("DROP TABLE " + identifier);
   }
 
+  @Test
+  public void testAlterTableAddColumns() throws Exception {
+    Assume.assumeTrue("Iceberg - alter table/add column is only relevant for HiveCatalog",
+        testTableType == TestTables.TestTableType.HIVE_CATALOG);
+
+    TableIdentifier identifier = TableIdentifier.of("default", "customers");
+
+    // Create HMS table with with a property to be translated
+    shell.executeStatement(String.format("CREATE EXTERNAL TABLE default.customers " +

Review comment:
       `TestTables` has several convenience methods to create tables with schema and spec. It might be useful here too




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] pvary commented on a change in pull request #2351: HIVE-25200: Alter table add columns support for Iceberg tables

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #2351:
URL: https://github.com/apache/hive/pull/2351#discussion_r645946009



##########
File path: iceberg/iceberg-catalog/src/main/java/org/apache/iceberg/hive/HiveSchemaUtil.java
##########
@@ -134,6 +136,28 @@ public static Type convert(TypeInfo typeInfo) {
     return HiveSchemaConverter.convert(typeInfo, false);
   }
 
+  /**
+   * Produces the difference of two FieldSchema lists by only taking into account the field name and type.
+   * @param from List of fields to subtract from
+   * @param to List of fields to subtract
+   * @return the result list of difference
+   */
+  public static List<FieldSchema> schemaDifference(List<FieldSchema> from, List<FieldSchema> to) {
+    List<FieldSchema> result = new LinkedList<>(from);

Review comment:
       Why use `List` as an input? `Collection` might serve as well.
   Why not use lambdas?
   Maybe just create a name/type map from the `to` list and just filter the results with a lambda? 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] szlta merged pull request #2351: HIVE-25200: Alter table add columns support for Iceberg tables

Posted by GitBox <gi...@apache.org>.
szlta merged pull request #2351:
URL: https://github.com/apache/hive/pull/2351


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] pvary commented on a change in pull request #2351: HIVE-25200: Alter table add columns support for Iceberg tables

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #2351:
URL: https://github.com/apache/hive/pull/2351#discussion_r645946917



##########
File path: iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerNoScan.java
##########
@@ -783,6 +785,43 @@ public void testDropHiveTableWithoutUnderlyingTable() throws IOException {
     shell.executeStatement("DROP TABLE " + identifier);
   }
 
+  @Test
+  public void testAlterTableAddColumns() throws Exception {
+    Assume.assumeTrue("Iceberg - alter table/add column is only relevant for HiveCatalog",

Review comment:
       We do not support ADD COLUMN for other catalogs? 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] szlta commented on pull request #2351: HIVE-25200: Alter table add columns support for Iceberg tables

Posted by GitBox <gi...@apache.org>.
szlta commented on pull request #2351:
URL: https://github.com/apache/hive/pull/2351#issuecomment-856626991


   > @szlta: quick question: Would it be possible to create a test where we concurrently try to modify the schema through Hive and change the schema through the Iceberg Java API?
   
   yep, added


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] marton-bod commented on a change in pull request #2351: HIVE-25200: Alter table add columns support for Iceberg tables

Posted by GitBox <gi...@apache.org>.
marton-bod commented on a change in pull request #2351:
URL: https://github.com/apache/hive/pull/2351#discussion_r646415970



##########
File path: iceberg/iceberg-catalog/src/main/java/org/apache/iceberg/hive/HiveSchemaUtil.java
##########
@@ -134,6 +136,28 @@ public static Type convert(TypeInfo typeInfo) {
     return HiveSchemaConverter.convert(typeInfo, false);
   }
 
+  /**
+   * Produces the difference of two FieldSchema lists by only taking into account the field name and type.
+   * @param from List of fields to subtract from
+   * @param to List of fields to subtract
+   * @return the result list of difference
+   */
+  public static List<FieldSchema> schemaDifference(List<FieldSchema> from, List<FieldSchema> to) {
+    List<FieldSchema> result = new LinkedList<>(from);
+    Iterator<FieldSchema> it = result.iterator();
+    while (it.hasNext()) {

Review comment:
       nit: maybe use Streams with filter and collect?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] marton-bod commented on a change in pull request #2351: HIVE-25200: Alter table add columns support for Iceberg tables

Posted by GitBox <gi...@apache.org>.
marton-bod commented on a change in pull request #2351:
URL: https://github.com/apache/hive/pull/2351#discussion_r646413108



##########
File path: iceberg/iceberg-catalog/src/main/java/org/apache/iceberg/hive/HiveSchemaUtil.java
##########
@@ -134,6 +136,28 @@ public static Type convert(TypeInfo typeInfo) {
     return HiveSchemaConverter.convert(typeInfo, false);
   }
 
+  /**
+   * Produces the difference of two FieldSchema lists by only taking into account the field name and type.
+   * @param from List of fields to subtract from
+   * @param to List of fields to subtract
+   * @return the result list of difference
+   */
+  public static List<FieldSchema> schemaDifference(List<FieldSchema> from, List<FieldSchema> to) {

Review comment:
       These parameter names suggest to me that `to` represent some destination collection, and `from` is the source collection. Can we rename them a bit? Maybe something like `baseFields` and `fieldsToRemove`?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] szlta commented on a change in pull request #2351: HIVE-25200: Alter table add columns support for Iceberg tables

Posted by GitBox <gi...@apache.org>.
szlta commented on a change in pull request #2351:
URL: https://github.com/apache/hive/pull/2351#discussion_r646608043



##########
File path: iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergMetaHook.java
##########
@@ -265,9 +287,12 @@ public void commitAlterTable(org.apache.hadoop.hive.metastore.api.Table hmsTable
       HiveTableUtil.importFiles(preAlterTableProperties.tableLocation, preAlterTableProperties.format,
           partitionSpecProxy, preAlterTableProperties.partitionKeys, catalogProperties, conf);
     } else {
-      Map<String, String> contextProperties = context.getProperties();
-      if (contextProperties.containsKey(ALTER_TABLE_OPERATION_TYPE) &&
-          allowedAlterTypes.contains(contextProperties.get(ALTER_TABLE_OPERATION_TYPE))) {
+      if (isMatchingAlterOp(AlterTableType.ADDCOLS, context) && updateSchema != null) {

Review comment:
       Ok this is now stored as a state of this hook.
   I usually like to be more restrictive, so I'd rather leave the ADDCOL op type check (btw I'm not sure how valid this case is, but if there are no new columns, then we have a case for ADDCOL op type with null updateSchema ;) )




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] pvary commented on pull request #2351: HIVE-25200: Alter table add columns support for Iceberg tables

Posted by GitBox <gi...@apache.org>.
pvary commented on pull request #2351:
URL: https://github.com/apache/hive/pull/2351#issuecomment-856166212


   @szlta: quick question: Would it be possible to create a test where we concurrently try to modify the schema through Hive and change the schema through the Iceberg Java API?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] szlta commented on a change in pull request #2351: HIVE-25200: Alter table add columns support for Iceberg tables

Posted by GitBox <gi...@apache.org>.
szlta commented on a change in pull request #2351:
URL: https://github.com/apache/hive/pull/2351#discussion_r647348841



##########
File path: iceberg/iceberg-catalog/src/main/java/org/apache/iceberg/hive/HiveSchemaUtil.java
##########
@@ -134,6 +137,23 @@ public static Type convert(TypeInfo typeInfo) {
     return HiveSchemaConverter.convert(typeInfo, false);
   }
 
+  /**
+   * Produces the difference of two FieldSchema lists by only taking into account the field name and type.
+   * @param subtrahendCollection List of fields to subtract from

Review comment:
       Woops, yeah..




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] pvary commented on a change in pull request #2351: HIVE-25200: Alter table add columns support for Iceberg tables

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #2351:
URL: https://github.com/apache/hive/pull/2351#discussion_r645947095



##########
File path: iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerNoScan.java
##########
@@ -783,6 +785,43 @@ public void testDropHiveTableWithoutUnderlyingTable() throws IOException {
     shell.executeStatement("DROP TABLE " + identifier);
   }
 
+  @Test
+  public void testAlterTableAddColumns() throws Exception {
+    Assume.assumeTrue("Iceberg - alter table/add column is only relevant for HiveCatalog",
+        testTableType == TestTables.TestTableType.HIVE_CATALOG);
+
+    TableIdentifier identifier = TableIdentifier.of("default", "customers");
+
+    // Create HMS table with with a property to be translated

Review comment:
       Maybe leftover comment? 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] marton-bod commented on a change in pull request #2351: HIVE-25200: Alter table add columns support for Iceberg tables

Posted by GitBox <gi...@apache.org>.
marton-bod commented on a change in pull request #2351:
URL: https://github.com/apache/hive/pull/2351#discussion_r647358340



##########
File path: iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergMetaHook.java
##########
@@ -310,6 +337,24 @@ public void rollbackAlterTable(org.apache.hadoop.hive.metastore.api.Table hmsTab
     }
   }
 
+  private void setupAlterOperationType(EnvironmentContext context) throws MetaException {
+    if (context != null) {
+      Map<String, String> contextProperties = context.getProperties();
+      if (contextProperties != null) {
+        String stringOpType = contextProperties.get(ALTER_TABLE_OPERATION_TYPE);
+        if (stringOpType != null) {
+          currentAlterTableOp = AlterTableType.valueOf(stringOpType);
+          if (SUPPORTED_ALTER_OPS.stream().noneMatch(op -> op.equals(currentAlterTableOp))) {
+            throw new MetaException(
+                "Unsupported ALTER TABLE operation type for Iceberg tables, must be: " + allowedAlterTypes.toString());
+          }
+        }
+        return;
+      }
+    }
+    throw new MetaException("ALTER TABLE operation type could not be determined.");

Review comment:
       Can we maybe get rid of the return by putting this exception to the beginning of the method?
   e.g. 
   ```
   if (context == null || context.getProperties() == null) {
      throw new ...
   }
   ```
   The other thing I'm thinking of is that it'd be informative to include the hmsTable name in the exception message as well (for this and the above too).




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] pvary commented on a change in pull request #2351: HIVE-25200: Alter table add columns support for Iceberg tables

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #2351:
URL: https://github.com/apache/hive/pull/2351#discussion_r647334605



##########
File path: iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergMetaHook.java
##########
@@ -310,6 +335,24 @@ public void rollbackAlterTable(org.apache.hadoop.hive.metastore.api.Table hmsTab
     }
   }
 
+  private void setupAlterOperationType(EnvironmentContext context) throws MetaException {
+    if (context != null) {
+      Map<String, String> contextProperties = context.getProperties();
+      if (contextProperties != null) {
+        String stringOpType = contextProperties.get(ALTER_TABLE_OPERATION_TYPE);
+        if (stringOpType != null) {
+          currentAlterTableOp = AlterTableType.valueOf(stringOpType);
+          if (SUPPORTED_ALTER_OPS.stream().noneMatch(op -> op.equals(currentAlterTableOp))) {
+            throw new MetaException(
+                "Unsupported ALTER TABLE operation type for Iceberg tables, must be: " + allowedAlterTypes.toString());
+          }
+        }
+        return;

Review comment:
       Why is this return here?
   Is it valid operation where `stringOpType` == null? What is the operation at that time?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] marton-bod commented on a change in pull request #2351: HIVE-25200: Alter table add columns support for Iceberg tables

Posted by GitBox <gi...@apache.org>.
marton-bod commented on a change in pull request #2351:
URL: https://github.com/apache/hive/pull/2351#discussion_r646423854



##########
File path: iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergMetaHook.java
##########
@@ -310,6 +335,26 @@ public void rollbackAlterTable(org.apache.hadoop.hive.metastore.api.Table hmsTab
     }
   }
 
+  private static boolean isSupportedAlterOperation(EnvironmentContext context) {
+    for (Enum<?> op : SUPPORTED_ALTER_OPS) {
+      if (isMatchingAlterOp(op, context)) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  private static boolean isMatchingAlterOp(Enum<?> alterOperation, EnvironmentContext context) {
+    if (context == null) {

Review comment:
       these two null checks could be extracted into `isSupportedAlterOperation`, since if they're null they'll always be false so no need for looping




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] pvary commented on a change in pull request #2351: HIVE-25200: Alter table add columns support for Iceberg tables

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #2351:
URL: https://github.com/apache/hive/pull/2351#discussion_r647332812



##########
File path: iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergMetaHook.java
##########
@@ -248,6 +256,20 @@ public void preAlterTable(org.apache.hadoop.hive.metastore.api.Table hmsTable, E
           Collections.emptyMap()));
       updateHmsTableProperties(hmsTable);
     }
+    if (AlterTableType.ADDCOLS.equals(currentAlterTableOp)) {

Review comment:
       nit: newline after block close




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] marton-bod commented on a change in pull request #2351: HIVE-25200: Alter table add columns support for Iceberg tables

Posted by GitBox <gi...@apache.org>.
marton-bod commented on a change in pull request #2351:
URL: https://github.com/apache/hive/pull/2351#discussion_r646433022



##########
File path: iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerNoScan.java
##########
@@ -783,6 +785,43 @@ public void testDropHiveTableWithoutUnderlyingTable() throws IOException {
     shell.executeStatement("DROP TABLE " + identifier);
   }
 
+  @Test
+  public void testAlterTableAddColumns() throws Exception {
+    Assume.assumeTrue("Iceberg - alter table/add column is only relevant for HiveCatalog",
+        testTableType == TestTables.TestTableType.HIVE_CATALOG);
+
+    TableIdentifier identifier = TableIdentifier.of("default", "customers");
+
+    // Create HMS table with with a property to be translated
+    shell.executeStatement(String.format("CREATE EXTERNAL TABLE default.customers " +
+            "STORED BY ICEBERG " +
+            "TBLPROPERTIES ('%s'='%s', '%s'='%s', '%s'='%s')",
+        InputFormatConfig.TABLE_SCHEMA, SchemaParser.toJson(HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA),
+        InputFormatConfig.PARTITION_SPEC, PartitionSpecParser.toJson(SPEC),
+        InputFormatConfig.EXTERNAL_TABLE_PURGE, "false"));
+
+    shell.executeStatement("ALTER TABLE default.customers ADD COLUMNS " +
+        "(newintcol int, newstringcol string COMMENT 'Column with description')");
+
+    org.apache.iceberg.Table icebergTable = testTables.loadTable(identifier);
+    org.apache.hadoop.hive.metastore.api.Table hmsTable = shell.metastore().getTable("default", "customers");
+
+    List<FieldSchema> icebergSchema = HiveSchemaUtil.convert(icebergTable.schema());
+    List<FieldSchema> hmsSchema = hmsTable.getSd().getCols();
+
+    List<FieldSchema> expectedSchema = Lists.newArrayList(
+        new FieldSchema("customer_id", "bigint", null),
+        new FieldSchema("first_name", "string", "This is first name"),
+        new FieldSchema("last_name", "string", "This is last name"),
+        new FieldSchema("newintcol", "int", null),
+        new FieldSchema("newstringcol", "string", "Column with description"));
+
+    Assert.assertEquals(expectedSchema, icebergSchema);
+    Assert.assertEquals(icebergSchema, hmsSchema);
+
+    shell.executeStatement("DROP TABLE " + identifier);

Review comment:
       You can remove this drop, since all tables are automatically dropped after each test case




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] pvary commented on a change in pull request #2351: HIVE-25200: Alter table add columns support for Iceberg tables

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #2351:
URL: https://github.com/apache/hive/pull/2351#discussion_r645947350



##########
File path: iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerNoScan.java
##########
@@ -783,6 +785,43 @@ public void testDropHiveTableWithoutUnderlyingTable() throws IOException {
     shell.executeStatement("DROP TABLE " + identifier);
   }
 
+  @Test
+  public void testAlterTableAddColumns() throws Exception {
+    Assume.assumeTrue("Iceberg - alter table/add column is only relevant for HiveCatalog",
+        testTableType == TestTables.TestTableType.HIVE_CATALOG);
+
+    TableIdentifier identifier = TableIdentifier.of("default", "customers");
+
+    // Create HMS table with with a property to be translated
+    shell.executeStatement(String.format("CREATE EXTERNAL TABLE default.customers " +
+            "STORED BY ICEBERG " +
+            "TBLPROPERTIES ('%s'='%s', '%s'='%s', '%s'='%s')",
+        InputFormatConfig.TABLE_SCHEMA, SchemaParser.toJson(HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA),
+        InputFormatConfig.PARTITION_SPEC, PartitionSpecParser.toJson(SPEC),
+        InputFormatConfig.EXTERNAL_TABLE_PURGE, "false"));
+
+    shell.executeStatement("ALTER TABLE default.customers ADD COLUMNS " +

Review comment:
       Do we want to insert and read data after the alter in another test? 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] marton-bod commented on a change in pull request #2351: HIVE-25200: Alter table add columns support for Iceberg tables

Posted by GitBox <gi...@apache.org>.
marton-bod commented on a change in pull request #2351:
URL: https://github.com/apache/hive/pull/2351#discussion_r646424725



##########
File path: iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergMetaHook.java
##########
@@ -310,6 +335,26 @@ public void rollbackAlterTable(org.apache.hadoop.hive.metastore.api.Table hmsTab
     }
   }
 
+  private static boolean isSupportedAlterOperation(EnvironmentContext context) {
+    for (Enum<?> op : SUPPORTED_ALTER_OPS) {

Review comment:
       nit maybe: `return SUPPORTED_ALTER_OPS.stream().anyMatch(op -> isMatchingAlterOp(op, context));`




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] marton-bod commented on a change in pull request #2351: HIVE-25200: Alter table add columns support for Iceberg tables

Posted by GitBox <gi...@apache.org>.
marton-bod commented on a change in pull request #2351:
URL: https://github.com/apache/hive/pull/2351#discussion_r647344554



##########
File path: iceberg/iceberg-catalog/src/main/java/org/apache/iceberg/hive/HiveSchemaUtil.java
##########
@@ -134,6 +137,23 @@ public static Type convert(TypeInfo typeInfo) {
     return HiveSchemaConverter.convert(typeInfo, false);
   }
 
+  /**
+   * Produces the difference of two FieldSchema lists by only taking into account the field name and type.
+   * @param subtrahendCollection List of fields to subtract from

Review comment:
       I think minuend and subtrahend are the other way around in this case, no?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] szlta commented on a change in pull request #2351: HIVE-25200: Alter table add columns support for Iceberg tables

Posted by GitBox <gi...@apache.org>.
szlta commented on a change in pull request #2351:
URL: https://github.com/apache/hive/pull/2351#discussion_r646608593



##########
File path: iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerNoScan.java
##########
@@ -783,6 +785,43 @@ public void testDropHiveTableWithoutUnderlyingTable() throws IOException {
     shell.executeStatement("DROP TABLE " + identifier);
   }
 
+  @Test
+  public void testAlterTableAddColumns() throws Exception {
+    Assume.assumeTrue("Iceberg - alter table/add column is only relevant for HiveCatalog",
+        testTableType == TestTables.TestTableType.HIVE_CATALOG);
+
+    TableIdentifier identifier = TableIdentifier.of("default", "customers");
+
+    // Create HMS table with with a property to be translated
+    shell.executeStatement(String.format("CREATE EXTERNAL TABLE default.customers " +
+            "STORED BY ICEBERG " +
+            "TBLPROPERTIES ('%s'='%s', '%s'='%s', '%s'='%s')",
+        InputFormatConfig.TABLE_SCHEMA, SchemaParser.toJson(HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA),
+        InputFormatConfig.PARTITION_SPEC, PartitionSpecParser.toJson(SPEC),
+        InputFormatConfig.EXTERNAL_TABLE_PURGE, "false"));
+
+    shell.executeStatement("ALTER TABLE default.customers ADD COLUMNS " +

Review comment:
       Yeah I'd refrain from doing a thorough check of that here




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] marton-bod commented on a change in pull request #2351: HIVE-25200: Alter table add columns support for Iceberg tables

Posted by GitBox <gi...@apache.org>.
marton-bod commented on a change in pull request #2351:
URL: https://github.com/apache/hive/pull/2351#discussion_r647358883



##########
File path: iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergMetaHook.java
##########
@@ -310,6 +335,24 @@ public void rollbackAlterTable(org.apache.hadoop.hive.metastore.api.Table hmsTab
     }
   }
 
+  private void setupAlterOperationType(EnvironmentContext context) throws MetaException {
+    if (context != null) {
+      Map<String, String> contextProperties = context.getProperties();
+      if (contextProperties != null) {
+        String stringOpType = contextProperties.get(ALTER_TABLE_OPERATION_TYPE);
+        if (stringOpType != null) {
+          currentAlterTableOp = AlterTableType.valueOf(stringOpType);
+          if (SUPPORTED_ALTER_OPS.stream().noneMatch(op -> op.equals(currentAlterTableOp))) {
+            throw new MetaException(
+                "Unsupported ALTER TABLE operation type for Iceberg tables, must be: " + allowedAlterTypes.toString());
+          }
+        }
+        return;

Review comment:
       Maybe a short comment explaining what you just said would be useful for future maintainers




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] marton-bod commented on a change in pull request #2351: HIVE-25200: Alter table add columns support for Iceberg tables

Posted by GitBox <gi...@apache.org>.
marton-bod commented on a change in pull request #2351:
URL: https://github.com/apache/hive/pull/2351#discussion_r646427858



##########
File path: iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergMetaHook.java
##########
@@ -213,7 +219,9 @@ public void commitDropTable(org.apache.hadoop.hive.metastore.api.Table hmsTable,
   @Override
   public void preAlterTable(org.apache.hadoop.hive.metastore.api.Table hmsTable, EnvironmentContext context)
       throws MetaException {
-    super.preAlterTable(hmsTable, context);
+    if (!isSupportedAlterOperation(context)) {
+      super.preAlterTable(hmsTable, context);

Review comment:
       Since we've added ADDPROPS and DROPPROPS to our allowed list as well, maybe we can remove the super call? It seems like it wouldn't add any additional value at this point?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] marton-bod commented on pull request #2351: HIVE-25200: Alter table add columns support for Iceberg tables

Posted by GitBox <gi...@apache.org>.
marton-bod commented on pull request #2351:
URL: https://github.com/apache/hive/pull/2351#issuecomment-860652825


   Rebased it here to run the tests again: https://github.com/apache/hive/pull/2392


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] pvary commented on a change in pull request #2351: HIVE-25200: Alter table add columns support for Iceberg tables

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #2351:
URL: https://github.com/apache/hive/pull/2351#discussion_r647356109



##########
File path: iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergMetaHook.java
##########
@@ -310,6 +335,24 @@ public void rollbackAlterTable(org.apache.hadoop.hive.metastore.api.Table hmsTab
     }
   }
 
+  private void setupAlterOperationType(EnvironmentContext context) throws MetaException {
+    if (context != null) {
+      Map<String, String> contextProperties = context.getProperties();
+      if (contextProperties != null) {
+        String stringOpType = contextProperties.get(ALTER_TABLE_OPERATION_TYPE);
+        if (stringOpType != null) {
+          currentAlterTableOp = AlterTableType.valueOf(stringOpType);
+          if (SUPPORTED_ALTER_OPS.stream().noneMatch(op -> op.equals(currentAlterTableOp))) {
+            throw new MetaException(
+                "Unsupported ALTER TABLE operation type for Iceberg tables, must be: " + allowedAlterTypes.toString());
+          }
+        }
+        return;

Review comment:
       I see. Maybe worth a comment then.
   Thanks for the explanation!




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] szlta merged pull request #2351: HIVE-25200: Alter table add columns support for Iceberg tables

Posted by GitBox <gi...@apache.org>.
szlta merged pull request #2351:
URL: https://github.com/apache/hive/pull/2351


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] marton-bod commented on a change in pull request #2351: HIVE-25200: Alter table add columns support for Iceberg tables

Posted by GitBox <gi...@apache.org>.
marton-bod commented on a change in pull request #2351:
URL: https://github.com/apache/hive/pull/2351#discussion_r646432618



##########
File path: iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerNoScan.java
##########
@@ -783,6 +785,43 @@ public void testDropHiveTableWithoutUnderlyingTable() throws IOException {
     shell.executeStatement("DROP TABLE " + identifier);
   }
 
+  @Test
+  public void testAlterTableAddColumns() throws Exception {
+    Assume.assumeTrue("Iceberg - alter table/add column is only relevant for HiveCatalog",
+        testTableType == TestTables.TestTableType.HIVE_CATALOG);
+
+    TableIdentifier identifier = TableIdentifier.of("default", "customers");
+
+    // Create HMS table with with a property to be translated
+    shell.executeStatement(String.format("CREATE EXTERNAL TABLE default.customers " +
+            "STORED BY ICEBERG " +
+            "TBLPROPERTIES ('%s'='%s', '%s'='%s', '%s'='%s')",
+        InputFormatConfig.TABLE_SCHEMA, SchemaParser.toJson(HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA),
+        InputFormatConfig.PARTITION_SPEC, PartitionSpecParser.toJson(SPEC),
+        InputFormatConfig.EXTERNAL_TABLE_PURGE, "false"));
+
+    shell.executeStatement("ALTER TABLE default.customers ADD COLUMNS " +

Review comment:
       There's a bit of an overlapping work on that here: https://jira.cloudera.com/browse/CDPD-25369, which should cover add/drop/rename column scenarios.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] pvary commented on a change in pull request #2351: HIVE-25200: Alter table add columns support for Iceberg tables

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #2351:
URL: https://github.com/apache/hive/pull/2351#discussion_r645946147



##########
File path: iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergMetaHook.java
##########
@@ -213,7 +219,9 @@ public void commitDropTable(org.apache.hadoop.hive.metastore.api.Table hmsTable,
   @Override
   public void preAlterTable(org.apache.hadoop.hive.metastore.api.Table hmsTable, EnvironmentContext context)
       throws MetaException {
-    super.preAlterTable(hmsTable, context);
+    if (!isSupportedAlterOperation(context)) {
+      super.preAlterTable(hmsTable, context);
+    }
     catalogProperties = getCatalogProperties(hmsTable);

Review comment:
       Nit: Iceberg reviewers ask for new line after closing a block




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] marton-bod commented on a change in pull request #2351: HIVE-25200: Alter table add columns support for Iceberg tables

Posted by GitBox <gi...@apache.org>.
marton-bod commented on a change in pull request #2351:
URL: https://github.com/apache/hive/pull/2351#discussion_r646436391



##########
File path: iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergMetaHook.java
##########
@@ -265,9 +287,12 @@ public void commitAlterTable(org.apache.hadoop.hive.metastore.api.Table hmsTable
       HiveTableUtil.importFiles(preAlterTableProperties.tableLocation, preAlterTableProperties.format,
           partitionSpecProxy, preAlterTableProperties.partitionKeys, catalogProperties, conf);
     } else {
-      Map<String, String> contextProperties = context.getProperties();
-      if (contextProperties.containsKey(ALTER_TABLE_OPERATION_TYPE) &&
-          allowedAlterTypes.contains(contextProperties.get(ALTER_TABLE_OPERATION_TYPE))) {
+      if (isMatchingAlterOp(AlterTableType.ADDCOLS, context) && updateSchema != null) {

Review comment:
       `updateSchema` can only be non-null if `isMatchingAlterOp(AlterTableType.ADDCOLS, context)` was true in preAlterTable, so maybe `if (updateSchema != null)` is enough here? That might make this part also work generically for other schema update operations, like drop and rename, since you just have to call `commit()` here regardless of the operation type.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] marton-bod commented on a change in pull request #2351: HIVE-25200: Alter table add columns support for Iceberg tables

Posted by GitBox <gi...@apache.org>.
marton-bod commented on a change in pull request #2351:
URL: https://github.com/apache/hive/pull/2351#discussion_r646424725



##########
File path: iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergMetaHook.java
##########
@@ -310,6 +335,26 @@ public void rollbackAlterTable(org.apache.hadoop.hive.metastore.api.Table hmsTab
     }
   }
 
+  private static boolean isSupportedAlterOperation(EnvironmentContext context) {
+    for (Enum<?> op : SUPPORTED_ALTER_OPS) {

Review comment:
       nit maybe: `return SUPPORTED_ALTER_OPS.stream().anyMatch(op -> isMatchingAlterOp(op, context));`
   
   or if the null checks are already here, it could even just be:
   
   `return SUPPORTED_ALTER_OPS.stream().anyMatch(op -> op.name().equals(contextProperties.get(ALTER_TABLE_OPERATION_TYPE));`




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] pvary commented on a change in pull request #2351: HIVE-25200: Alter table add columns support for Iceberg tables

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #2351:
URL: https://github.com/apache/hive/pull/2351#discussion_r645946749



##########
File path: iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergMetaHook.java
##########
@@ -265,9 +287,12 @@ public void commitAlterTable(org.apache.hadoop.hive.metastore.api.Table hmsTable
       HiveTableUtil.importFiles(preAlterTableProperties.tableLocation, preAlterTableProperties.format,
           partitionSpecProxy, preAlterTableProperties.partitionKeys, catalogProperties, conf);
     } else {
-      Map<String, String> contextProperties = context.getProperties();
-      if (contextProperties.containsKey(ALTER_TABLE_OPERATION_TYPE) &&
-          allowedAlterTypes.contains(contextProperties.get(ALTER_TABLE_OPERATION_TYPE))) {
+      if (isMatchingAlterOp(AlterTableType.ADDCOLS, context) && updateSchema != null) {

Review comment:
       Wouldn't it be cleaner to just get the operation in the beginning and use that through the method. I find it confusing that we call `isMatchingAlterOp` all the time




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org