[ 
https://issues.apache.org/jira/browse/HIVE-25234?focusedWorklogId=610121&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-610121
 ]

ASF GitHub Bot logged work on HIVE-25234:
-----------------------------------------

                Author: ASF GitHub Bot
            Created on: 14/Jun/21 07:19
            Start Date: 14/Jun/21 07:19
    Worklog Time Spent: 10m 
      Work Description: marton-bod commented on a change in pull request #2382:
URL: https://github.com/apache/hive/pull/2382#discussion_r649853361



##########
File path: 
iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergMetaHook.java
##########
@@ -311,6 +312,38 @@ public void 
rollbackAlterTable(org.apache.hadoop.hive.metastore.api.Table hmsTab
     }
   }
 
+  private void alterTableProperties(org.apache.hadoop.hive.metastore.api.Table 
hmsTable,
+      Map<String, String> contextProperties) {
+    Map<String, String> hmsTableParameters = hmsTable.getParameters();
+    Splitter splitter = Splitter.on(PROPERTIES_SEPARATOR);
+    UpdateProperties icebergUpdateProperties = icebergTable.updateProperties();
+    if (contextProperties.containsKey(SET_PROPERTIES)) {
+      splitter.splitToList(contextProperties.get(SET_PROPERTIES))
+          .forEach(k -> icebergUpdateProperties.set(k, 
hmsTableParameters.get(k)));

Review comment:
       I know it's not part of this PR's topic, but I just noticed: shouldn't 
we do the hms-iceberg property key translation here? instead of just `k` -> 
`HiveTableOperations.translateToIcebergProp(k)`

##########
File path: 
iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/IcebergTableUtil.java
##########
@@ -106,4 +111,61 @@ public static PartitionSpec spec(Configuration 
configuration, Schema schema) {
     });
     return builder.build();
   }
+
+  public static void updateSpec(Configuration configuration, Table table) {
+    // get the new partition transform spec
+    PartitionSpec newPartitionSpec = spec(configuration, table.schema());
+    if (newPartitionSpec == null) {
+      LOG.debug("Iceberg Partition spec is not updated due to empty partition 
spec definition.");

Review comment:
       Is this scenario possible? Would this happen if you ran `ALTER TABLE tbl 
SET PARTITION SPEC()`? The parser would not complain about this?

##########
File path: 
iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/IcebergTableUtil.java
##########
@@ -106,4 +111,61 @@ public static PartitionSpec spec(Configuration 
configuration, Schema schema) {
     });
     return builder.build();
   }
+
+  public static void updateSpec(Configuration configuration, Table table) {
+    // get the new partition transform spec
+    PartitionSpec newPartitionSpec = spec(configuration, table.schema());
+    if (newPartitionSpec == null) {
+      LOG.debug("Iceberg Partition spec is not updated due to empty partition 
spec definition.");
+      return;
+    }
+
+    List<PartitionTransform.PartitionTransformSpec> partitionTransformSpecList 
= SessionStateUtil
+        .getResource(configuration, 
hive_metastoreConstants.PARTITION_TRANSFORM_SPEC)
+        .map(o -> (List<PartitionTransform.PartitionTransformSpec>) 
o).orElseGet(() -> null);
+    List<String> newPartitionNames =
+        
newPartitionSpec.fields().stream().map(PartitionField::name).collect(Collectors.toList());
+    List<String> currentPartitionNames = 
table.spec().fields().stream().map(PartitionField::name)
+        .collect(Collectors.toList());
+    List<String> intersectingPartitionNames =
+        
currentPartitionNames.stream().filter(newPartitionNames::contains).collect(Collectors.toList());
+
+    // delete those partitions which are not present among the new partion spec
+    UpdatePartitionSpec updatePartitionSpec = table.updateSpec();
+    currentPartitionNames.stream().filter(p -> 
!intersectingPartitionNames.contains(p))
+        .forEach(updatePartitionSpec::removeField);
+    updatePartitionSpec.apply();
+
+    // add new partitions which are not yet present
+    IntStream.range(0, partitionTransformSpecList.size())

Review comment:
       Could we loop through partitionTransformSpecList directly without the 
IntStream?
   Maybe this is possible?
   ```
   partitionTransformSpecList.stream()
      .filter(spec -> !intersectingPartitionNames.contains(spec.name)
      .forEach(...)
   ```

##########
File path: 
iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/IcebergTableUtil.java
##########
@@ -106,4 +111,61 @@ public static PartitionSpec spec(Configuration 
configuration, Schema schema) {
     });
     return builder.build();
   }
+
+  public static void updateSpec(Configuration configuration, Table table) {
+    // get the new partition transform spec
+    PartitionSpec newPartitionSpec = spec(configuration, table.schema());
+    if (newPartitionSpec == null) {
+      LOG.debug("Iceberg Partition spec is not updated due to empty partition 
spec definition.");
+      return;
+    }
+
+    List<PartitionTransform.PartitionTransformSpec> partitionTransformSpecList 
= SessionStateUtil

Review comment:
       nit: can you move this closer to its place of usage, the part where we 
add the new specs?

##########
File path: 
iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/IcebergTableUtil.java
##########
@@ -106,4 +111,61 @@ public static PartitionSpec spec(Configuration 
configuration, Schema schema) {
     });
     return builder.build();
   }
+
+  public static void updateSpec(Configuration configuration, Table table) {
+    // get the new partition transform spec
+    PartitionSpec newPartitionSpec = spec(configuration, table.schema());
+    if (newPartitionSpec == null) {
+      LOG.debug("Iceberg Partition spec is not updated due to empty partition 
spec definition.");
+      return;
+    }
+
+    List<PartitionTransform.PartitionTransformSpec> partitionTransformSpecList 
= SessionStateUtil
+        .getResource(configuration, 
hive_metastoreConstants.PARTITION_TRANSFORM_SPEC)
+        .map(o -> (List<PartitionTransform.PartitionTransformSpec>) 
o).orElseGet(() -> null);
+    List<String> newPartitionNames =
+        
newPartitionSpec.fields().stream().map(PartitionField::name).collect(Collectors.toList());
+    List<String> currentPartitionNames = 
table.spec().fields().stream().map(PartitionField::name)
+        .collect(Collectors.toList());
+    List<String> intersectingPartitionNames =
+        
currentPartitionNames.stream().filter(newPartitionNames::contains).collect(Collectors.toList());
+
+    // delete those partitions which are not present among the new partion spec
+    UpdatePartitionSpec updatePartitionSpec = table.updateSpec();
+    currentPartitionNames.stream().filter(p -> 
!intersectingPartitionNames.contains(p))
+        .forEach(updatePartitionSpec::removeField);
+    updatePartitionSpec.apply();
+
+    // add new partitions which are not yet present
+    IntStream.range(0, partitionTransformSpecList.size())

Review comment:
       I guess not, because spec.name would only contain the column name, so 
never mind :)

##########
File path: 
iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerNoScan.java
##########
@@ -155,6 +155,58 @@ public void after() throws Exception {
     HiveIcebergStorageHandlerTestUtils.close(shell);
   }
 
+  @Test
+  public void testSetPartitionTransform() {
+    Schema schema = new Schema(
+        optional(1, "id", Types.LongType.get()),
+        optional(2, "year_field", Types.DateType.get()),
+        optional(3, "month_field", Types.TimestampType.withZone()),
+        optional(4, "day_field", Types.TimestampType.withoutZone()),
+        optional(5, "hour_field", Types.TimestampType.withoutZone()),
+        optional(6, "truncate_field", Types.StringType.get()),
+        optional(7, "bucket_field", Types.StringType.get()),
+        optional(8, "identity_field", Types.StringType.get())
+    );
+
+    TableIdentifier identifier = TableIdentifier.of("default", "part_test");
+    shell.executeStatement("CREATE EXTERNAL TABLE " + identifier +
+        " PARTITIONED BY SPEC (year(year_field), hour(hour_field), " +
+        "truncate(2, truncate_field), bucket(2, bucket_field), 
identity_field)" +
+        " STORED BY ICEBERG " +
+        testTables.locationForCreateTableSQL(identifier) +
+        "TBLPROPERTIES ('" + InputFormatConfig.TABLE_SCHEMA + "'='" +
+        SchemaParser.toJson(schema) + "', " +
+        "'" + InputFormatConfig.CATALOG_NAME + "'='" + 
Catalogs.ICEBERG_DEFAULT_CATALOG_NAME + "')");
+
+    PartitionSpec spec = PartitionSpec.builderFor(schema)
+        .year("year_field")
+        .hour("hour_field")
+        .truncate("truncate_field", 2)
+        .bucket("bucket_field", 2)
+        .identity("identity_field")
+        .build();
+
+    Table table = testTables.loadTable(identifier);
+    Assert.assertEquals(spec, table.spec());
+
+    shell.executeStatement("ALTER TABLE default.part_test SET PARTITION 
SPEC(year(year_field), month(month_field), " +
+        "day(day_field))");
+
+    spec = PartitionSpec.builderFor(schema)
+        .withSpecId(1)
+        .year("year_field")
+        .alwaysNull("hour_field", "hour_field_hour")
+        .alwaysNull("truncate_field", "truncate_field_trunc")
+        .alwaysNull("bucket_field", "bucket_field_bucket")
+        .alwaysNull("identity_field", "identity_field")
+        .month("month_field")
+        .day("day_field")
+        .build();
+
+    table.refresh();
+    Assert.assertEquals(spec, table.spec());
+  }
+

Review comment:
       I'm thinking of two other test scenarios:
   
   - Changing the partition transform type on the same column. E.g. month(ts) 
-> day(ts). In the current test, we have separate columns like month_field or 
day_field.
   - Making an unpartitioned table partitioned, by calling this alter table on 
it. I'm not 100% sure this is supported, but I could theoretically image it 
working (and users trying it out).
   
   What do you think?

##########
File path: 
iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerNoScan.java
##########
@@ -155,6 +155,58 @@ public void after() throws Exception {
     HiveIcebergStorageHandlerTestUtils.close(shell);
   }
 
+  @Test
+  public void testSetPartitionTransform() {
+    Schema schema = new Schema(
+        optional(1, "id", Types.LongType.get()),
+        optional(2, "year_field", Types.DateType.get()),
+        optional(3, "month_field", Types.TimestampType.withZone()),
+        optional(4, "day_field", Types.TimestampType.withoutZone()),
+        optional(5, "hour_field", Types.TimestampType.withoutZone()),
+        optional(6, "truncate_field", Types.StringType.get()),
+        optional(7, "bucket_field", Types.StringType.get()),
+        optional(8, "identity_field", Types.StringType.get())
+    );
+
+    TableIdentifier identifier = TableIdentifier.of("default", "part_test");
+    shell.executeStatement("CREATE EXTERNAL TABLE " + identifier +
+        " PARTITIONED BY SPEC (year(year_field), hour(hour_field), " +
+        "truncate(2, truncate_field), bucket(2, bucket_field), 
identity_field)" +
+        " STORED BY ICEBERG " +
+        testTables.locationForCreateTableSQL(identifier) +
+        "TBLPROPERTIES ('" + InputFormatConfig.TABLE_SCHEMA + "'='" +
+        SchemaParser.toJson(schema) + "', " +
+        "'" + InputFormatConfig.CATALOG_NAME + "'='" + 
Catalogs.ICEBERG_DEFAULT_CATALOG_NAME + "')");
+
+    PartitionSpec spec = PartitionSpec.builderFor(schema)
+        .year("year_field")
+        .hour("hour_field")
+        .truncate("truncate_field", 2)
+        .bucket("bucket_field", 2)
+        .identity("identity_field")
+        .build();
+
+    Table table = testTables.loadTable(identifier);
+    Assert.assertEquals(spec, table.spec());
+
+    shell.executeStatement("ALTER TABLE default.part_test SET PARTITION 
SPEC(year(year_field), month(month_field), " +
+        "day(day_field))");
+
+    spec = PartitionSpec.builderFor(schema)
+        .withSpecId(1)
+        .year("year_field")
+        .alwaysNull("hour_field", "hour_field_hour")
+        .alwaysNull("truncate_field", "truncate_field_trunc")
+        .alwaysNull("bucket_field", "bucket_field_bucket")
+        .alwaysNull("identity_field", "identity_field")
+        .month("month_field")
+        .day("day_field")
+        .build();
+
+    table.refresh();
+    Assert.assertEquals(spec, table.spec());
+  }
+

Review comment:
       I'm thinking of two other test scenarios:
   
   - Changing the partition transform type on the same column. E.g. month(ts) 
-> day(ts). In the current test, we have separate columns for each, like 
month_field or day_field, so would be great to try changing the same column too.
   - Making an unpartitioned table partitioned, by calling this alter table on 
it. I'm not 100% sure this is supported, but I could theoretically imagine it 
working (and users trying it out :).
   
   What do you think?




-- 
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:
us...@infra.apache.org


Issue Time Tracking
-------------------

    Worklog Id:     (was: 610121)
    Time Spent: 0.5h  (was: 20m)

> Implement ALTER TABLE ... SET PARTITION SPEC to change partitioning on 
> Iceberg tables
> -------------------------------------------------------------------------------------
>
>                 Key: HIVE-25234
>                 URL: https://issues.apache.org/jira/browse/HIVE-25234
>             Project: Hive
>          Issue Type: Improvement
>            Reporter: László Pintér
>            Assignee: László Pintér
>            Priority: Major
>              Labels: pull-request-available
>          Time Spent: 0.5h
>  Remaining Estimate: 0h
>
> Provide a way to change the schema and the Iceberg partitioning specification 
> using Hive syntax.
> {code:sql}
> ALTER TABLE tbl SET PARTITION SPEC(...)
> {code}



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to