szehon-ho commented on code in PR #12319:
URL: https://github.com/apache/iceberg/pull/12319#discussion_r1961116327


##########
spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java:
##########
@@ -1085,4 +1088,43 @@ private ExecutorService getService() {
       return service;
     }
   }
+
+  /**
+   * Returns the first partition spec in an IcebergTable that shares the same 
names and ordering as
+   * the partition columns in a given Spark Table. Throws an error if not found
+   */
+  private static PartitionSpec findCompatibleSpec(
+      Table icebergTable, SparkSession spark, String sparkTable) throws 
AnalysisException {
+    List<String> parts = 
Lists.newArrayList(Splitter.on('.').limit(2).split(sparkTable));
+    String db = parts.size() == 1 ? "default" : parts.get(0);
+    String table = parts.get(parts.size() == 1 ? 0 : 1);
+
+    List<String> sparkPartNames =
+        spark.catalog().listColumns(db, table).collectAsList().stream()
+            .filter(org.apache.spark.sql.catalog.Column::isPartition)
+            .map(org.apache.spark.sql.catalog.Column::name)
+            .map(name -> name.toLowerCase(Locale.ROOT))
+            .collect(Collectors.toList());
+
+    for (PartitionSpec icebergSpec : icebergTable.specs().values()) {
+      boolean allIdentity =
+          icebergSpec.fields().stream().allMatch(field -> 
field.transform().isIdentity());
+      if (allIdentity) {
+        List<String> icebergPartNames =
+            icebergSpec.fields().stream()
+                .map(PartitionField::name)
+                .map(name -> name.toLowerCase(Locale.ROOT))
+                .collect(Collectors.toList());
+        if (icebergPartNames.equals(sparkPartNames)) {
+          return icebergSpec;
+        }
+      }
+    }
+
+    throw new IllegalArgumentException(
+        String.format(
+            "Cannot find a partition spec in Iceberg table %s that matches the 
partition"
+                + "columns (%s) in Spark table %s",

Review Comment:
   Nit: is this missing a front space ?



##########
spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java:
##########
@@ -476,6 +485,53 @@ public void 
addPartitionToPartitionedSnapshotIdInheritanceEnabledInTwoRuns() {
         sql("SELECT id, name, dept, subdept FROM %s WHERE id < 3 ORDER BY id", 
sourceTableName),
         sql("SELECT id, name, dept, subdept FROM %s ORDER BY id", tableName));
 
+    Table table = Spark3Util.loadIcebergTable(spark, tableName);
+    FileIO io = ((HasTableOperations) table).operations().io();
+
+    assertThat(
+            table.currentSnapshot().allManifests(io).stream()
+                .map(mf -> ManifestFiles.read(mf, io, null /* force reading 
spec from file*/))
+                .collect(Collectors.toList()))
+        .allMatch(file -> file.spec().equals(table.spec()));
+
+    // verify manifest file name has uuid pattern
+    String manifestPath = (String) sql("select path from %s.manifests", 
tableName).get(0)[0];
+
+    Pattern uuidPattern = 
Pattern.compile("[a-f0-9]{8}(?:-[a-f0-9]{4}){4}[a-f0-9]{8}");
+
+    Matcher matcher = uuidPattern.matcher(manifestPath);
+    assertThat(matcher.find()).as("verify manifest path has uuid").isTrue();
+  }
+
+  @TestTemplate
+  public void addPartitionsFromHiveSnapshotInheritanceEnabled()
+      throws NoSuchTableException, ParseException {
+    createPartitionedHiveTable();
+    createIcebergTable(
+        "id Integer, name String, dept String, subdept String", "PARTITIONED 
BY (id)");
+
+    sql(
+        "ALTER TABLE %s SET TBLPROPERTIES ('%s' 'true')",
+        tableName, TableProperties.SNAPSHOT_ID_INHERITANCE_ENABLED);
+
+    sql("CALL %s.system.add_files('%s', '%s')", catalogName, tableName, 
sourceTableName);
+
+    assertEquals(
+        "Iceberg table contains correct data",
+        sql("SELECT id, name, dept, subdept FROM %s ORDER BY id", 
sourceTableName),
+        sql("SELECT id, name, dept, subdept FROM %s ORDER BY id", tableName));
+
+    Table table = Spark3Util.loadIcebergTable(spark, tableName);
+    FileIO io = ((HasTableOperations) table).operations().io();
+
+    // Check that the manifest written have the correct partition spec written

Review Comment:
   Nit: last 'written' seems redundant?



##########
spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java:
##########
@@ -476,6 +485,53 @@ public void 
addPartitionToPartitionedSnapshotIdInheritanceEnabledInTwoRuns() {
         sql("SELECT id, name, dept, subdept FROM %s WHERE id < 3 ORDER BY id", 
sourceTableName),
         sql("SELECT id, name, dept, subdept FROM %s ORDER BY id", tableName));
 
+    Table table = Spark3Util.loadIcebergTable(spark, tableName);
+    FileIO io = ((HasTableOperations) table).operations().io();
+
+    assertThat(
+            table.currentSnapshot().allManifests(io).stream()
+                .map(mf -> ManifestFiles.read(mf, io, null /* force reading 
spec from file*/))
+                .collect(Collectors.toList()))
+        .allMatch(file -> file.spec().equals(table.spec()));
+
+    // verify manifest file name has uuid pattern

Review Comment:
   Maybe we can refactor this strange uuid check into its own method as its now 
in two 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: issues-unsubscr...@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org
For additional commands, e-mail: issues-h...@iceberg.apache.org

Reply via email to