You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by GitBox <gi...@apache.org> on 2022/06/04 05:20:02 UTC

[GitHub] [iceberg] CodingCat commented on a diff in pull request #4956: support adding extra commit metadata with SQL in Spark

CodingCat commented on code in PR #4956:
URL: https://github.com/apache/iceberg/pull/4956#discussion_r889486223


##########
spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java:
##########
@@ -404,4 +414,61 @@ public void testExtraSnapshotMetadata() throws IOException {
     Assert.assertTrue(table.currentSnapshot().summary().get("extra-key").equals("someValue"));
     Assert.assertTrue(table.currentSnapshot().summary().get("another-key").equals("anotherValue"));
   }
+
+  @Test
+  public void testExtraSnapshotMetadataWithSQL() throws IOException {
+    String tableLocation = temp.newFolder("iceberg-table").toString();
+    HadoopTables tables = new HadoopTables(CONF);
+    int threadsCount = 3;
+    ExecutorService executorService = Executors.newFixedThreadPool(threadsCount, new ThreadFactory() {

Review Comment:
   updated to ThreadPool, I think multi-threading testing is still necessary? as we need to have something guarding that the commit metadata change is thread safe no matter we use ThreadLocal as now or later we change to something else for any reason 
   



##########
spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java:
##########
@@ -404,4 +414,61 @@ public void testExtraSnapshotMetadata() throws IOException {
     Assert.assertTrue(table.currentSnapshot().summary().get("extra-key").equals("someValue"));
     Assert.assertTrue(table.currentSnapshot().summary().get("another-key").equals("anotherValue"));
   }
+
+  @Test
+  public void testExtraSnapshotMetadataWithSQL() throws IOException {
+    String tableLocation = temp.newFolder("iceberg-table").toString();
+    HadoopTables tables = new HadoopTables(CONF);
+    int threadsCount = 3;
+    ExecutorService executorService = Executors.newFixedThreadPool(threadsCount, new ThreadFactory() {
+
+      private AtomicInteger currentThreadCount = new AtomicInteger(0);
+
+      @Override
+      public Thread newThread(Runnable r) {
+        return new Thread(r, "thread-" + currentThreadCount.getAndIncrement());
+      }
+    });
+
+    Table table = tables.create(SCHEMA, PartitionSpec.unpartitioned(), Maps.newHashMap(), tableLocation);
+
+    List<SimpleRecord> expectedRecords = Lists.newArrayList(
+        new SimpleRecord(1, "a"),
+        new SimpleRecord(2, "b")
+    );
+    Dataset<Row> originalDf = spark.createDataFrame(expectedRecords, SimpleRecord.class);
+    originalDf.select("id", "data").write()
+        .format("iceberg")
+        .mode("append")
+        .option(SparkWriteOptions.SNAPSHOT_PROPERTY_PREFIX + ".extra-key", "someValue")
+        .option(SparkWriteOptions.SNAPSHOT_PROPERTY_PREFIX + ".another-key", "anotherValue")
+        .save(tableLocation);
+    spark.read().format("iceberg").load(tableLocation).createOrReplaceTempView("target");
+    Tasks
+        .range(threadsCount)
+        .stopOnFailure()
+        .throwFailureWhenFinished()
+        .executeWith(executorService)
+        .run(index -> {
+              Map<String, String> properties = Maps.newHashMap();
+              properties.put("writer-thread", String.valueOf(Thread.currentThread().getName()));
+              try {
+                CommitMetadata.withCommitProperties(properties, () -> {
+                  spark.sql("INSERT INTO target VALUES (3, 'c'), (4, 'd')");
+                  return 0;
+                });
+              } catch (Exception e) {
+                e.printStackTrace();
+              }
+            }
+        );
+    Set<String> threadNames = new HashSet<>();

Review Comment:
   updated
   



-- 
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-unsubscribe@iceberg.apache.org

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


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