You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hudi.apache.org by GitBox <gi...@apache.org> on 2022/10/06 23:17:14 UTC

[GitHub] [hudi] alexeykudinkin opened a new pull request, #6883: [HUDI-4992] Fixing invalid min/max record key stats in Parquet metadata

alexeykudinkin opened a new pull request, #6883:
URL: https://github.com/apache/hudi/pull/6883

   ### Change Logs
   
   Troubleshooting duplicates issue, we've found that the min/max record key stats are being currently persisted incorrectly into Parquet metadata by Row-writing Bulk Insert, leading to duplicate records upon using the Bloom Filter subsequently.
   
   ### Impact
   
   **Risk level: Low**
   
   ### Documentation Update
   
   No docs update
   
   ### Contributor's checklist
   
   - [ ] Read through [contributor's guide](https://hudi.apache.org/contribute/how-to-contribute)
   - [ ] Change Logs and Impact were stated clearly
   - [ ] Adequate tests were added if applicable
   - [ ] CI passed
   


-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] alexeykudinkin commented on pull request #6883: [HUDI-4992] Fixing invalid min/max record key stats in Parquet metadata

Posted by GitBox <gi...@apache.org>.
alexeykudinkin commented on PR #6883:
URL: https://github.com/apache/hudi/pull/6883#issuecomment-1270940646

   @nsivabalan yes, i'm adding comprehensive tests for Parquet writers to make sure we properly validate both records and metadata written to Parquet


-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] alexeykudinkin commented on a diff in pull request #6883: [HUDI-4992] Fixing invalid min/max record key stats in Parquet metadata

Posted by GitBox <gi...@apache.org>.
alexeykudinkin commented on code in PR #6883:
URL: https://github.com/apache/hudi/pull/6883#discussion_r989616904


##########
hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/storage/row/TestHoodieInternalRowParquetWriter.java:
##########
@@ -68,44 +72,55 @@ public void tearDown() throws Exception {
 
   @ParameterizedTest
   @ValueSource(booleans = {true, false})
-  public void endToEndTest(boolean parquetWriteLegacyFormatEnabled) throws Exception {
+  public void testProperWriting(boolean parquetWriteLegacyFormatEnabled) throws Exception {
+    // Generate inputs
+    Dataset<Row> inputRows = SparkDatasetTestUtils.getRandomRows(sqlContext, 100,
+        HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, false);
+    StructType schema = inputRows.schema();
+
+    List<InternalRow> rows = SparkDatasetTestUtils.toInternalRows(inputRows, SparkDatasetTestUtils.ENCODER);
+
     HoodieWriteConfig.Builder writeConfigBuilder =
         SparkDatasetTestUtils.getConfigBuilder(basePath, timelineServicePort);
-    for (int i = 0; i < 5; i++) {
-      // init write support and parquet config
-      HoodieRowParquetWriteSupport writeSupport = getWriteSupport(writeConfigBuilder, hadoopConf, parquetWriteLegacyFormatEnabled);
-      HoodieWriteConfig cfg = writeConfigBuilder.build();
-      HoodieParquetConfig<HoodieRowParquetWriteSupport> parquetConfig = new HoodieParquetConfig<>(writeSupport,
-          CompressionCodecName.SNAPPY, cfg.getParquetBlockSize(), cfg.getParquetPageSize(), cfg.getParquetMaxFileSize(),
-          writeSupport.getHadoopConf(), cfg.getParquetCompressionRatio(), cfg.parquetDictionaryEnabled());
-
-      // prepare path
-      String fileId = UUID.randomUUID().toString();
-      Path filePath = new Path(basePath + "/" + fileId);
-      String partitionPath = HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH;
-      metaClient.getFs().mkdirs(new Path(basePath));
-
-      // init writer
-      HoodieInternalRowParquetWriter writer = new HoodieInternalRowParquetWriter(filePath, parquetConfig);
-
-      // generate input
-      int size = 10 + RANDOM.nextInt(100);
-      // Generate inputs
-      Dataset<Row> inputRows = SparkDatasetTestUtils.getRandomRows(sqlContext, size, partitionPath, false);
-      List<InternalRow> internalRows = SparkDatasetTestUtils.toInternalRows(inputRows, SparkDatasetTestUtils.ENCODER);
-
-      // issue writes
-      for (InternalRow internalRow : internalRows) {
-        writer.write(internalRow);
-      }
 
-      // close the writer
-      writer.close();
+    HoodieRowParquetWriteSupport writeSupport = getWriteSupport(writeConfigBuilder, hadoopConf, parquetWriteLegacyFormatEnabled);
+    HoodieWriteConfig cfg = writeConfigBuilder.build();
+    HoodieParquetConfig<HoodieRowParquetWriteSupport> parquetConfig = new HoodieParquetConfig<>(writeSupport,
+        CompressionCodecName.SNAPPY, cfg.getParquetBlockSize(), cfg.getParquetPageSize(), cfg.getParquetMaxFileSize(),

Review Comment:
   I'm carrying over this setup from the previous test and frankly i don't think the codec we use matter much in this test, but (practically) in every other test we actually test against the codec that is set in our config by default (since we're writing parquet in most of them)



-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] hudi-bot commented on pull request #6883: [HUDI-4992] Fixing invalid min/max record key stats in Parquet metadata

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6883:
URL: https://github.com/apache/hudi/pull/6883#issuecomment-1270857238

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "024bcf68a222e98835e0b61fa784bc1040f65f62",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12037",
       "triggerID" : "024bcf68a222e98835e0b61fa784bc1040f65f62",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 024bcf68a222e98835e0b61fa784bc1040f65f62 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12037) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] nsivabalan commented on pull request #6883: [HUDI-4992] Fixing invalid min/max record key stats in Parquet metadata

Posted by GitBox <gi...@apache.org>.
nsivabalan commented on PR #6883:
URL: https://github.com/apache/hudi/pull/6883#issuecomment-1270864726

   can we please add tests as well


-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] nsivabalan commented on a diff in pull request #6883: [HUDI-4992] Fixing invalid min/max record key stats in Parquet metadata

Posted by GitBox <gi...@apache.org>.
nsivabalan commented on code in PR #6883:
URL: https://github.com/apache/hudi/pull/6883#discussion_r989615334


##########
hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/storage/row/TestHoodieInternalRowParquetWriter.java:
##########
@@ -68,44 +72,55 @@ public void tearDown() throws Exception {
 
   @ParameterizedTest
   @ValueSource(booleans = {true, false})
-  public void endToEndTest(boolean parquetWriteLegacyFormatEnabled) throws Exception {
+  public void testProperWriting(boolean parquetWriteLegacyFormatEnabled) throws Exception {
+    // Generate inputs
+    Dataset<Row> inputRows = SparkDatasetTestUtils.getRandomRows(sqlContext, 100,
+        HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, false);
+    StructType schema = inputRows.schema();
+
+    List<InternalRow> rows = SparkDatasetTestUtils.toInternalRows(inputRows, SparkDatasetTestUtils.ENCODER);
+
     HoodieWriteConfig.Builder writeConfigBuilder =
         SparkDatasetTestUtils.getConfigBuilder(basePath, timelineServicePort);
-    for (int i = 0; i < 5; i++) {
-      // init write support and parquet config
-      HoodieRowParquetWriteSupport writeSupport = getWriteSupport(writeConfigBuilder, hadoopConf, parquetWriteLegacyFormatEnabled);
-      HoodieWriteConfig cfg = writeConfigBuilder.build();
-      HoodieParquetConfig<HoodieRowParquetWriteSupport> parquetConfig = new HoodieParquetConfig<>(writeSupport,
-          CompressionCodecName.SNAPPY, cfg.getParquetBlockSize(), cfg.getParquetPageSize(), cfg.getParquetMaxFileSize(),
-          writeSupport.getHadoopConf(), cfg.getParquetCompressionRatio(), cfg.parquetDictionaryEnabled());
-
-      // prepare path
-      String fileId = UUID.randomUUID().toString();
-      Path filePath = new Path(basePath + "/" + fileId);
-      String partitionPath = HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH;
-      metaClient.getFs().mkdirs(new Path(basePath));
-
-      // init writer
-      HoodieInternalRowParquetWriter writer = new HoodieInternalRowParquetWriter(filePath, parquetConfig);
-
-      // generate input
-      int size = 10 + RANDOM.nextInt(100);
-      // Generate inputs
-      Dataset<Row> inputRows = SparkDatasetTestUtils.getRandomRows(sqlContext, size, partitionPath, false);
-      List<InternalRow> internalRows = SparkDatasetTestUtils.toInternalRows(inputRows, SparkDatasetTestUtils.ENCODER);
-
-      // issue writes
-      for (InternalRow internalRow : internalRows) {
-        writer.write(internalRow);
-      }
 
-      // close the writer
-      writer.close();
+    HoodieRowParquetWriteSupport writeSupport = getWriteSupport(writeConfigBuilder, hadoopConf, parquetWriteLegacyFormatEnabled);
+    HoodieWriteConfig cfg = writeConfigBuilder.build();
+    HoodieParquetConfig<HoodieRowParquetWriteSupport> parquetConfig = new HoodieParquetConfig<>(writeSupport,
+        CompressionCodecName.SNAPPY, cfg.getParquetBlockSize(), cfg.getParquetPageSize(), cfg.getParquetMaxFileSize(),

Review Comment:
   can we use default compression based on our config values. so that when we switch our default sometime, these tests will test those codec



-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] hudi-bot commented on pull request #6883: [HUDI-4992] Fixing invalid min/max record key stats in Parquet metadata

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6883:
URL: https://github.com/apache/hudi/pull/6883#issuecomment-1271075750

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "024bcf68a222e98835e0b61fa784bc1040f65f62",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12037",
       "triggerID" : "024bcf68a222e98835e0b61fa784bc1040f65f62",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c528116cecfd35edc4b6829cb091c4924dc1758f",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12038",
       "triggerID" : "c528116cecfd35edc4b6829cb091c4924dc1758f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c528116cecfd35edc4b6829cb091c4924dc1758f Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12038) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] hudi-bot commented on pull request #6883: [HUDI-4992] Fixing invalid min/max record key stats in Parquet metadata

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6883:
URL: https://github.com/apache/hudi/pull/6883#issuecomment-1270854312

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "024bcf68a222e98835e0b61fa784bc1040f65f62",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "024bcf68a222e98835e0b61fa784bc1040f65f62",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 024bcf68a222e98835e0b61fa784bc1040f65f62 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] hudi-bot commented on pull request #6883: [HUDI-4992] Fixing invalid min/max record key stats in Parquet metadata

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6883:
URL: https://github.com/apache/hudi/pull/6883#issuecomment-1270989045

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "024bcf68a222e98835e0b61fa784bc1040f65f62",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12037",
       "triggerID" : "024bcf68a222e98835e0b61fa784bc1040f65f62",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c528116cecfd35edc4b6829cb091c4924dc1758f",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12038",
       "triggerID" : "c528116cecfd35edc4b6829cb091c4924dc1758f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 024bcf68a222e98835e0b61fa784bc1040f65f62 Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12037) 
   * c528116cecfd35edc4b6829cb091c4924dc1758f Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12038) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] hudi-bot commented on pull request #6883: [HUDI-4992] Fixing invalid min/max record key stats in Parquet metadata

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6883:
URL: https://github.com/apache/hudi/pull/6883#issuecomment-1270982443

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "024bcf68a222e98835e0b61fa784bc1040f65f62",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12037",
       "triggerID" : "024bcf68a222e98835e0b61fa784bc1040f65f62",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c528116cecfd35edc4b6829cb091c4924dc1758f",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c528116cecfd35edc4b6829cb091c4924dc1758f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 024bcf68a222e98835e0b61fa784bc1040f65f62 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12037) 
   * c528116cecfd35edc4b6829cb091c4924dc1758f UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] xushiyan merged pull request #6883: [HUDI-4992] Fixing invalid min/max record key stats in Parquet metadata

Posted by GitBox <gi...@apache.org>.
xushiyan merged PR #6883:
URL: https://github.com/apache/hudi/pull/6883


-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] nsivabalan commented on a diff in pull request #6883: [HUDI-4992] Fixing invalid min/max record key stats in Parquet metadata

Posted by GitBox <gi...@apache.org>.
nsivabalan commented on code in PR #6883:
URL: https://github.com/apache/hudi/pull/6883#discussion_r989564188


##########
hudi-common/src/main/java/org/apache/hudi/avro/HoodieBloomFilterWriteSupport.java:
##########
@@ -0,0 +1,78 @@
+package org.apache.hudi.avro;
+
+import org.apache.hudi.common.bloom.BloomFilter;
+import org.apache.hudi.common.bloom.HoodieDynamicBoundedBloomFilter;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.apache.hudi.avro.HoodieAvroWriteSupport.HOODIE_AVRO_BLOOM_FILTER_METADATA_KEY;
+
+/**
+ * This is write-support utility base-class taking up handling of
+ *
+ * <ul>
+ *   <li>Adding record keys to the Bloom Filter</li>
+ *   <li>Keeping track of min/max record key (w/in single file)</li>
+ * </ul>
+ *
+ * @param <T> record-key type being ingested by this clas
+ */
+public abstract class HoodieBloomFilterWriteSupport<T extends Comparable<T>> {
+
+  public static final String HOODIE_MIN_RECORD_KEY_FOOTER = "hoodie_min_record_key";
+  public static final String HOODIE_MAX_RECORD_KEY_FOOTER = "hoodie_max_record_key";
+  public static final String HOODIE_BLOOM_FILTER_TYPE_CODE = "hoodie_bloom_filter_type_code";
+
+  private final BloomFilter bloomFilter;
+
+  private T minRecordKey;
+  private T maxRecordKey;
+
+  public HoodieBloomFilterWriteSupport(BloomFilter bloomFilter) {
+    this.bloomFilter = bloomFilter;
+  }
+
+  public void addKey(T recordKey) {
+    bloomFilter.add(getUTF8Bytes(recordKey));

Review Comment:
   previously, we were adding strings as is into bloom filter (HoodieAvroWriteSupport), but now w/ this patch, we are switching to adding UTF8Bytes? 



-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] alexeykudinkin commented on a diff in pull request #6883: [HUDI-4992] Fixing invalid min/max record key stats in Parquet metadata

Posted by GitBox <gi...@apache.org>.
alexeykudinkin commented on code in PR #6883:
URL: https://github.com/apache/hudi/pull/6883#discussion_r989585629


##########
hudi-common/src/main/java/org/apache/hudi/avro/HoodieBloomFilterWriteSupport.java:
##########
@@ -0,0 +1,78 @@
+package org.apache.hudi.avro;
+
+import org.apache.hudi.common.bloom.BloomFilter;
+import org.apache.hudi.common.bloom.HoodieDynamicBoundedBloomFilter;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.apache.hudi.avro.HoodieAvroWriteSupport.HOODIE_AVRO_BLOOM_FILTER_METADATA_KEY;
+
+/**
+ * This is write-support utility base-class taking up handling of
+ *
+ * <ul>
+ *   <li>Adding record keys to the Bloom Filter</li>
+ *   <li>Keeping track of min/max record key (w/in single file)</li>
+ * </ul>
+ *
+ * @param <T> record-key type being ingested by this clas
+ */
+public abstract class HoodieBloomFilterWriteSupport<T extends Comparable<T>> {
+
+  public static final String HOODIE_MIN_RECORD_KEY_FOOTER = "hoodie_min_record_key";
+  public static final String HOODIE_MAX_RECORD_KEY_FOOTER = "hoodie_max_record_key";
+  public static final String HOODIE_BLOOM_FILTER_TYPE_CODE = "hoodie_bloom_filter_type_code";
+
+  private final BloomFilter bloomFilter;
+
+  private T minRecordKey;
+  private T maxRecordKey;
+
+  public HoodieBloomFilterWriteSupport(BloomFilter bloomFilter) {
+    this.bloomFilter = bloomFilter;
+  }
+
+  public void addKey(T recordKey) {
+    bloomFilter.add(getUTF8Bytes(recordKey));

Review Comment:
   Bloom Filter always adds keys as UTF8, though previously we're just passing Strings into it (except in Row-writer)



-- 
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: commits-unsubscribe@hudi.apache.org

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