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/10/12 12:09:50 UTC

[GitHub] [hive] pvary opened a new pull request #2715: HIVE-25610: Handle partition field comments for Iceberg tables

pvary opened a new pull request #2715:
URL: https://github.com/apache/hive/pull/2715


   ### What changes were proposed in this pull request?
   Fixing partition column comments handling for Iceberg tables.
   Also AbstractSerDe provides methods to parse the column related fields from the properties. We should reuse them whenever possible.
   
   ### Why are the changes needed?
   Fixing the comment issue, and cleaning the code
   
   ### Does this PR introduce _any_ user-facing change?
   Fixes the issue
   
   ### How was this patch tested?
   Added unit tests


-- 
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.

To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org

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 #2715: HIVE-25610: Handle partition field comments for Iceberg tables

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



##########
File path: iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergSerDe.java
##########
@@ -19,10 +19,9 @@
 
 package org.apache.iceberg.mr.hive;
 
-import java.util.ArrayList;
+import com.google.common.collect.Lists;

Review comment:
       Done




-- 
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.

To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org

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 #2715: HIVE-25610: Handle partition field comments for Iceberg tables

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



##########
File path: iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergSerDe.java
##########
@@ -231,44 +223,28 @@ public ObjectInspector getObjectInspector() {
   /**
    * Gets the hive schema from the serDeProperties, and throws an exception if it is not provided. In the later case

Review comment:
       Done. Could you please check the woring?

##########
File path: iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergSerDe.java
##########
@@ -231,44 +223,28 @@ public ObjectInspector getObjectInspector() {
   /**
    * Gets the hive schema from the serDeProperties, and throws an exception if it is not provided. In the later case

Review comment:
       Done. Could you please check the wording?




-- 
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.

To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org

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 #2715: HIVE-25610: Handle partition field comments for Iceberg tables

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



##########
File path: iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergSerDe.java
##########
@@ -19,31 +19,27 @@
 
 package org.apache.iceberg.mr.hive;
 
-import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
-import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Properties;
 import java.util.stream.Collectors;
 import java.util.stream.IntStream;
 import javax.annotation.Nullable;
+import com.google.common.collect.Lists;

Review comment:
       Good catch. Fixed!




-- 
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.

To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org

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 #2715: HIVE-25610: Handle partition field comments for Iceberg tables

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



##########
File path: iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergSerDe.java
##########
@@ -19,31 +19,27 @@
 
 package org.apache.iceberg.mr.hive;
 
-import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
-import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Properties;
 import java.util.stream.Collectors;
 import java.util.stream.IntStream;
 import javax.annotation.Nullable;
+import com.google.common.collect.Lists;

Review comment:
       Don't we use the relocated guava in the iceberg classes?

##########
File path: iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergSerDe.java
##########
@@ -231,44 +223,28 @@ public ObjectInspector getObjectInspector() {
   /**
    * Gets the hive schema from the serDeProperties, and throws an exception if it is not provided. In the later case

Review comment:
       Let's modify the javadocs too according to the new logic

##########
File path: iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergSerDe.java
##########
@@ -19,10 +19,9 @@
 
 package org.apache.iceberg.mr.hive;
 
-import java.util.ArrayList;
+import com.google.common.collect.Lists;

Review comment:
       Do we want to use the relocated version of guava?

##########
File path: iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerNoScan.java
##########
@@ -721,6 +721,29 @@ public void testCreateTableWithoutColumnComments() {
     }
   }
 
+  @Test
+  public void testCreatePartitionedTableWithColumnComments() {
+    TableIdentifier identifier = TableIdentifier.of("default", "partitioned_with_comment_table");
+    shell.executeStatement("CREATE EXTERNAL TABLE partitioned_with_comment_table (" +
+        "t_int INT COMMENT 'int column',  " +
+        "t_string STRING COMMENT 'string column', " +
+        "t_string_2 STRING) " +
+        "PARTITIONED BY (t_string_3 STRING COMMENT 'partition column') " +
+        "STORED BY ICEBERG " +
+        testTables.locationForCreateTableSQL(identifier) +
+        testTables.propertiesForCreateTableSQL(ImmutableMap.of()));
+    org.apache.iceberg.Table icebergTable = testTables.loadTable(identifier);
+
+    List<Object[]> rows = shell.executeStatement("DESCRIBE default.partitioned_with_comment_table");
+    // The partition transform information is 3 extra lines, and 1 more line for the columns
+    Assert.assertEquals(icebergTable.schema().columns().size() + 4, rows.size());
+    for (int i = 0; i < icebergTable.schema().columns().size(); i++) {
+      Types.NestedField field = icebergTable.schema().columns().get(i);

Review comment:
       nit: can we extract the `icebergTable.schema().columns()` into a variable

##########
File path: iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergSerDe.java
##########
@@ -231,44 +223,28 @@ public ObjectInspector getObjectInspector() {
   /**
    * Gets the hive schema from the serDeProperties, and throws an exception if it is not provided. In the later case
    * it adds the previousException as a root cause.
-   * @param serDeProperties The source of the hive schema
    * @param previousException If we had an exception previously
    * @param autoConversion When <code>true</code>, convert unsupported types to more permissive ones, like tinyint to
    *                       int
    * @return The hive schema parsed from the serDeProperties
    * @throws SerDeException If there is no schema information in the serDeProperties
    */
-  private static Schema hiveSchemaOrThrow(Properties serDeProperties, Exception previousException,
-                                          boolean autoConversion)
+  private Schema hiveSchemaOrThrow(Exception previousException, boolean autoConversion)
       throws SerDeException {
-    // Read the configuration parameters
-    String columnNames = serDeProperties.getProperty(serdeConstants.LIST_COLUMNS);
-    String columnTypes = serDeProperties.getProperty(serdeConstants.LIST_COLUMN_TYPES);
-    // No constant for column comments and column comments delimiter.
-    String columnComments = serDeProperties.getProperty(LIST_COLUMN_COMMENT);
-    String columnNameDelimiter = serDeProperties.containsKey(serdeConstants.COLUMN_NAME_DELIMITER) ?
-        serDeProperties.getProperty(serdeConstants.COLUMN_NAME_DELIMITER) : String.valueOf(SerDeUtils.COMMA);
-    if (columnNames != null && columnTypes != null && columnNameDelimiter != null &&
-        !columnNames.isEmpty() && !columnTypes.isEmpty() && !columnNameDelimiter.isEmpty()) {
-      // Parse the configuration parameters
-      List<String> names = new ArrayList<>();
-      Collections.addAll(names, columnNames.split(columnNameDelimiter));
-      // check if there are partition columns as well
-      String partColNames = serDeProperties.getProperty(serdeConstants.LIST_PARTITION_COLUMNS);
-      if (partColNames != null && !partColNames.isEmpty()) {
-        // add partition col names to regular col names
-        String partColDelimiter = partColNames.contains(String.valueOf(ColumnType.COLUMN_COMMENTS_DELIMITER)) ?
-            String.valueOf(ColumnType.COLUMN_COMMENTS_DELIMITER) : String.valueOf(SerDeUtils.COMMA);
-        Collections.addAll(names, partColNames.split(partColDelimiter));
-        // add partition col types to regular col types
-        columnTypes += SerDeUtils.COLON + serDeProperties.getProperty(serdeConstants.LIST_PARTITION_COLUMN_TYPES);
-      }
-      List<String> comments = new ArrayList<>();
-      if (columnComments != null) {
-        Collections.addAll(comments, columnComments.split(Character.toString(Character.MIN_VALUE)));
-      }
-      Schema hiveSchema = HiveSchemaUtil.convert(names, TypeInfoUtils.getTypeInfosFromTypeString(columnTypes),
-              comments, autoConversion);
+    List<String> names = Lists.newArrayList();

Review comment:
       So much cleaner, nice




-- 
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.

To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org

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 merged pull request #2715: HIVE-25610: Handle partition field comments for Iceberg tables

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


   


-- 
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.

To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org

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 #2715: HIVE-25610: Handle partition field comments for Iceberg tables

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



##########
File path: iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergSerDe.java
##########
@@ -19,31 +19,27 @@
 
 package org.apache.iceberg.mr.hive;
 
-import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
-import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Properties;
 import java.util.stream.Collectors;
 import java.util.stream.IntStream;
 import javax.annotation.Nullable;
+import com.google.common.collect.Lists;

Review comment:
       Good catch. Fixed!

##########
File path: iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergSerDe.java
##########
@@ -231,44 +223,28 @@ public ObjectInspector getObjectInspector() {
   /**
    * Gets the hive schema from the serDeProperties, and throws an exception if it is not provided. In the later case

Review comment:
       Done. Could you please check the woring?

##########
File path: iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergSerDe.java
##########
@@ -231,44 +223,28 @@ public ObjectInspector getObjectInspector() {
   /**
    * Gets the hive schema from the serDeProperties, and throws an exception if it is not provided. In the later case

Review comment:
       Done. Could you please check the wording?

##########
File path: iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergSerDe.java
##########
@@ -231,44 +223,28 @@ public ObjectInspector getObjectInspector() {
   /**
    * Gets the hive schema from the serDeProperties, and throws an exception if it is not provided. In the later case
    * it adds the previousException as a root cause.
-   * @param serDeProperties The source of the hive schema
    * @param previousException If we had an exception previously
    * @param autoConversion When <code>true</code>, convert unsupported types to more permissive ones, like tinyint to
    *                       int
    * @return The hive schema parsed from the serDeProperties
    * @throws SerDeException If there is no schema information in the serDeProperties
    */
-  private static Schema hiveSchemaOrThrow(Properties serDeProperties, Exception previousException,
-                                          boolean autoConversion)
+  private Schema hiveSchemaOrThrow(Exception previousException, boolean autoConversion)
       throws SerDeException {
-    // Read the configuration parameters
-    String columnNames = serDeProperties.getProperty(serdeConstants.LIST_COLUMNS);
-    String columnTypes = serDeProperties.getProperty(serdeConstants.LIST_COLUMN_TYPES);
-    // No constant for column comments and column comments delimiter.
-    String columnComments = serDeProperties.getProperty(LIST_COLUMN_COMMENT);
-    String columnNameDelimiter = serDeProperties.containsKey(serdeConstants.COLUMN_NAME_DELIMITER) ?
-        serDeProperties.getProperty(serdeConstants.COLUMN_NAME_DELIMITER) : String.valueOf(SerDeUtils.COMMA);
-    if (columnNames != null && columnTypes != null && columnNameDelimiter != null &&
-        !columnNames.isEmpty() && !columnTypes.isEmpty() && !columnNameDelimiter.isEmpty()) {
-      // Parse the configuration parameters
-      List<String> names = new ArrayList<>();
-      Collections.addAll(names, columnNames.split(columnNameDelimiter));
-      // check if there are partition columns as well
-      String partColNames = serDeProperties.getProperty(serdeConstants.LIST_PARTITION_COLUMNS);
-      if (partColNames != null && !partColNames.isEmpty()) {
-        // add partition col names to regular col names
-        String partColDelimiter = partColNames.contains(String.valueOf(ColumnType.COLUMN_COMMENTS_DELIMITER)) ?
-            String.valueOf(ColumnType.COLUMN_COMMENTS_DELIMITER) : String.valueOf(SerDeUtils.COMMA);
-        Collections.addAll(names, partColNames.split(partColDelimiter));
-        // add partition col types to regular col types
-        columnTypes += SerDeUtils.COLON + serDeProperties.getProperty(serdeConstants.LIST_PARTITION_COLUMN_TYPES);
-      }
-      List<String> comments = new ArrayList<>();
-      if (columnComments != null) {
-        Collections.addAll(comments, columnComments.split(Character.toString(Character.MIN_VALUE)));
-      }
-      Schema hiveSchema = HiveSchemaUtil.convert(names, TypeInfoUtils.getTypeInfosFromTypeString(columnTypes),
-              comments, autoConversion);
+    List<String> names = Lists.newArrayList();

Review comment:
       Thx 😄 

##########
File path: iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerNoScan.java
##########
@@ -721,6 +721,29 @@ public void testCreateTableWithoutColumnComments() {
     }
   }
 
+  @Test
+  public void testCreatePartitionedTableWithColumnComments() {
+    TableIdentifier identifier = TableIdentifier.of("default", "partitioned_with_comment_table");
+    shell.executeStatement("CREATE EXTERNAL TABLE partitioned_with_comment_table (" +
+        "t_int INT COMMENT 'int column',  " +
+        "t_string STRING COMMENT 'string column', " +
+        "t_string_2 STRING) " +
+        "PARTITIONED BY (t_string_3 STRING COMMENT 'partition column') " +
+        "STORED BY ICEBERG " +
+        testTables.locationForCreateTableSQL(identifier) +
+        testTables.propertiesForCreateTableSQL(ImmutableMap.of()));
+    org.apache.iceberg.Table icebergTable = testTables.loadTable(identifier);
+
+    List<Object[]> rows = shell.executeStatement("DESCRIBE default.partitioned_with_comment_table");
+    // The partition transform information is 3 extra lines, and 1 more line for the columns
+    Assert.assertEquals(icebergTable.schema().columns().size() + 4, rows.size());
+    for (int i = 0; i < icebergTable.schema().columns().size(); i++) {
+      Types.NestedField field = icebergTable.schema().columns().get(i);

Review comment:
       Done

##########
File path: iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergSerDe.java
##########
@@ -19,10 +19,9 @@
 
 package org.apache.iceberg.mr.hive;
 
-import java.util.ArrayList;
+import com.google.common.collect.Lists;

Review comment:
       Done

##########
File path: iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerNoScan.java
##########
@@ -721,6 +721,29 @@ public void testCreateTableWithoutColumnComments() {
     }
   }
 
+  @Test
+  public void testCreatePartitionedTableWithColumnComments() {
+    TableIdentifier identifier = TableIdentifier.of("default", "partitioned_with_comment_table");
+    shell.executeStatement("CREATE EXTERNAL TABLE partitioned_with_comment_table (" +
+        "t_int INT COMMENT 'int column',  " +
+        "t_string STRING COMMENT 'string column', " +
+        "t_string_2 STRING) " +
+        "PARTITIONED BY (t_string_3 STRING COMMENT 'partition column') " +
+        "STORED BY ICEBERG " +
+        testTables.locationForCreateTableSQL(identifier) +
+        testTables.propertiesForCreateTableSQL(ImmutableMap.of()));
+    org.apache.iceberg.Table icebergTable = testTables.loadTable(identifier);
+
+    List<Object[]> rows = shell.executeStatement("DESCRIBE default.partitioned_with_comment_table");
+    // The partition transform information is 3 extra lines, and 1 more line for the columns
+    Assert.assertEquals(icebergTable.schema().columns().size() + 4, rows.size());
+    for (int i = 0; i < icebergTable.schema().columns().size(); i++) {
+      Types.NestedField field = icebergTable.schema().columns().get(i);
+      Assert.assertArrayEquals(new Object[] {field.name(), HiveSchemaUtil.convert(field.type()).getTypeName(),
+          field.doc() != null ? field.doc() : "from deserializer"}, rows.get(i));

Review comment:
       Added tests




-- 
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.

To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org

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 #2715: HIVE-25610: Handle partition field comments for Iceberg tables

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



##########
File path: iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerNoScan.java
##########
@@ -721,6 +721,29 @@ public void testCreateTableWithoutColumnComments() {
     }
   }
 
+  @Test
+  public void testCreatePartitionedTableWithColumnComments() {
+    TableIdentifier identifier = TableIdentifier.of("default", "partitioned_with_comment_table");
+    shell.executeStatement("CREATE EXTERNAL TABLE partitioned_with_comment_table (" +
+        "t_int INT COMMENT 'int column',  " +
+        "t_string STRING COMMENT 'string column', " +
+        "t_string_2 STRING) " +
+        "PARTITIONED BY (t_string_3 STRING COMMENT 'partition column') " +
+        "STORED BY ICEBERG " +
+        testTables.locationForCreateTableSQL(identifier) +
+        testTables.propertiesForCreateTableSQL(ImmutableMap.of()));
+    org.apache.iceberg.Table icebergTable = testTables.loadTable(identifier);
+
+    List<Object[]> rows = shell.executeStatement("DESCRIBE default.partitioned_with_comment_table");
+    // The partition transform information is 3 extra lines, and 1 more line for the columns
+    Assert.assertEquals(icebergTable.schema().columns().size() + 4, rows.size());
+    for (int i = 0; i < icebergTable.schema().columns().size(); i++) {
+      Types.NestedField field = icebergTable.schema().columns().get(i);
+      Assert.assertArrayEquals(new Object[] {field.name(), HiveSchemaUtil.convert(field.type()).getTypeName(),
+          field.doc() != null ? field.doc() : "from deserializer"}, rows.get(i));

Review comment:
       Do we know for sure that the Iceberg schema has the partition column comment pushed down? In case it's null on both Hive and Iceberg side, we'd still pass the test 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.

To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org

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 merged pull request #2715: HIVE-25610: Handle partition field comments for Iceberg tables

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


   


-- 
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.

To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org

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 #2715: HIVE-25610: Handle partition field comments for Iceberg tables

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



##########
File path: iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergSerDe.java
##########
@@ -231,44 +223,28 @@ public ObjectInspector getObjectInspector() {
   /**
    * Gets the hive schema from the serDeProperties, and throws an exception if it is not provided. In the later case

Review comment:
       Looks good




-- 
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.

To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org

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 #2715: HIVE-25610: Handle partition field comments for Iceberg tables

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



##########
File path: iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergSerDe.java
##########
@@ -231,44 +223,28 @@ public ObjectInspector getObjectInspector() {
   /**
    * Gets the hive schema from the serDeProperties, and throws an exception if it is not provided. In the later case
    * it adds the previousException as a root cause.
-   * @param serDeProperties The source of the hive schema
    * @param previousException If we had an exception previously
    * @param autoConversion When <code>true</code>, convert unsupported types to more permissive ones, like tinyint to
    *                       int
    * @return The hive schema parsed from the serDeProperties
    * @throws SerDeException If there is no schema information in the serDeProperties
    */
-  private static Schema hiveSchemaOrThrow(Properties serDeProperties, Exception previousException,
-                                          boolean autoConversion)
+  private Schema hiveSchemaOrThrow(Exception previousException, boolean autoConversion)
       throws SerDeException {
-    // Read the configuration parameters
-    String columnNames = serDeProperties.getProperty(serdeConstants.LIST_COLUMNS);
-    String columnTypes = serDeProperties.getProperty(serdeConstants.LIST_COLUMN_TYPES);
-    // No constant for column comments and column comments delimiter.
-    String columnComments = serDeProperties.getProperty(LIST_COLUMN_COMMENT);
-    String columnNameDelimiter = serDeProperties.containsKey(serdeConstants.COLUMN_NAME_DELIMITER) ?
-        serDeProperties.getProperty(serdeConstants.COLUMN_NAME_DELIMITER) : String.valueOf(SerDeUtils.COMMA);
-    if (columnNames != null && columnTypes != null && columnNameDelimiter != null &&
-        !columnNames.isEmpty() && !columnTypes.isEmpty() && !columnNameDelimiter.isEmpty()) {
-      // Parse the configuration parameters
-      List<String> names = new ArrayList<>();
-      Collections.addAll(names, columnNames.split(columnNameDelimiter));
-      // check if there are partition columns as well
-      String partColNames = serDeProperties.getProperty(serdeConstants.LIST_PARTITION_COLUMNS);
-      if (partColNames != null && !partColNames.isEmpty()) {
-        // add partition col names to regular col names
-        String partColDelimiter = partColNames.contains(String.valueOf(ColumnType.COLUMN_COMMENTS_DELIMITER)) ?
-            String.valueOf(ColumnType.COLUMN_COMMENTS_DELIMITER) : String.valueOf(SerDeUtils.COMMA);
-        Collections.addAll(names, partColNames.split(partColDelimiter));
-        // add partition col types to regular col types
-        columnTypes += SerDeUtils.COLON + serDeProperties.getProperty(serdeConstants.LIST_PARTITION_COLUMN_TYPES);
-      }
-      List<String> comments = new ArrayList<>();
-      if (columnComments != null) {
-        Collections.addAll(comments, columnComments.split(Character.toString(Character.MIN_VALUE)));
-      }
-      Schema hiveSchema = HiveSchemaUtil.convert(names, TypeInfoUtils.getTypeInfosFromTypeString(columnTypes),
-              comments, autoConversion);
+    List<String> names = Lists.newArrayList();

Review comment:
       Thx 😄 




-- 
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.

To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org

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 #2715: HIVE-25610: Handle partition field comments for Iceberg tables

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



##########
File path: iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergSerDe.java
##########
@@ -231,44 +223,28 @@ public ObjectInspector getObjectInspector() {
   /**
    * Gets the hive schema from the serDeProperties, and throws an exception if it is not provided. In the later case

Review comment:
       Looks good




-- 
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.

To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org

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 #2715: HIVE-25610: Handle partition field comments for Iceberg tables

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



##########
File path: iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerNoScan.java
##########
@@ -721,6 +721,29 @@ public void testCreateTableWithoutColumnComments() {
     }
   }
 
+  @Test
+  public void testCreatePartitionedTableWithColumnComments() {
+    TableIdentifier identifier = TableIdentifier.of("default", "partitioned_with_comment_table");
+    shell.executeStatement("CREATE EXTERNAL TABLE partitioned_with_comment_table (" +
+        "t_int INT COMMENT 'int column',  " +
+        "t_string STRING COMMENT 'string column', " +
+        "t_string_2 STRING) " +
+        "PARTITIONED BY (t_string_3 STRING COMMENT 'partition column') " +
+        "STORED BY ICEBERG " +
+        testTables.locationForCreateTableSQL(identifier) +
+        testTables.propertiesForCreateTableSQL(ImmutableMap.of()));
+    org.apache.iceberg.Table icebergTable = testTables.loadTable(identifier);
+
+    List<Object[]> rows = shell.executeStatement("DESCRIBE default.partitioned_with_comment_table");
+    // The partition transform information is 3 extra lines, and 1 more line for the columns
+    Assert.assertEquals(icebergTable.schema().columns().size() + 4, rows.size());
+    for (int i = 0; i < icebergTable.schema().columns().size(); i++) {
+      Types.NestedField field = icebergTable.schema().columns().get(i);

Review comment:
       Done




-- 
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.

To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org

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 #2715: HIVE-25610: Handle partition field comments for Iceberg tables

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



##########
File path: iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerNoScan.java
##########
@@ -721,6 +721,29 @@ public void testCreateTableWithoutColumnComments() {
     }
   }
 
+  @Test
+  public void testCreatePartitionedTableWithColumnComments() {
+    TableIdentifier identifier = TableIdentifier.of("default", "partitioned_with_comment_table");
+    shell.executeStatement("CREATE EXTERNAL TABLE partitioned_with_comment_table (" +
+        "t_int INT COMMENT 'int column',  " +
+        "t_string STRING COMMENT 'string column', " +
+        "t_string_2 STRING) " +
+        "PARTITIONED BY (t_string_3 STRING COMMENT 'partition column') " +
+        "STORED BY ICEBERG " +
+        testTables.locationForCreateTableSQL(identifier) +
+        testTables.propertiesForCreateTableSQL(ImmutableMap.of()));
+    org.apache.iceberg.Table icebergTable = testTables.loadTable(identifier);
+
+    List<Object[]> rows = shell.executeStatement("DESCRIBE default.partitioned_with_comment_table");
+    // The partition transform information is 3 extra lines, and 1 more line for the columns
+    Assert.assertEquals(icebergTable.schema().columns().size() + 4, rows.size());
+    for (int i = 0; i < icebergTable.schema().columns().size(); i++) {
+      Types.NestedField field = icebergTable.schema().columns().get(i);
+      Assert.assertArrayEquals(new Object[] {field.name(), HiveSchemaUtil.convert(field.type()).getTypeName(),
+          field.doc() != null ? field.doc() : "from deserializer"}, rows.get(i));

Review comment:
       Added tests




-- 
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.

To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org

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