boneanxs commented on code in PR #7362:
URL: https://github.com/apache/hudi/pull/7362#discussion_r1138237833


##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BaseSparkCommitActionExecutor.java:
##########
@@ -418,4 +426,22 @@ public Partitioner getLayoutPartitioner(WorkloadProfile 
profile, String layoutPa
   protected void 
runPrecommitValidators(HoodieWriteMetadata<HoodieData<WriteStatus>> 
writeMetadata) {
     SparkValidatorUtils.runValidators(config, writeMetadata, context, table, 
instantTime);
   }
+
+  private int dynamicSampleRecordSize(JavaRDD<HoodieRecord<T>> inputRecords) {
+    int dynamicSampleRecordSize = config.getCopyOnWriteRecordSizeEstimate();
+    int maxSampleRecordNum =  config.getRecordSizeDynamicSamplingMaxnum();
+    try {
+      List<HoodieRecord<T>> sampleRecords = inputRecords.takeSample(false, 
maxSampleRecordNum);
+      if (sampleRecords.size() == 0) {
+        LOG.warn("SampleRecords is empty.");

Review Comment:
   This log is not specific and meaningless.



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java:
##########
@@ -560,6 +560,20 @@ public class HoodieWriteConfig extends HoodieConfig {
       .withDocumentation("When table is upgraded from pre 0.12 to 0.12, we 
check for \"default\" partition and fail if found one. "
           + "Users are expected to rewrite the data in those partitions. 
Enabling this config will bypass this validation");
 
+  public static final ConfigProperty<String> 
RECORD_SIZE_DYNAMIC_SAMPLING_MAXNUM = ConfigProperty
+          .key("hoodie.record.size.dynamic.sampling.maxnum")
+          .defaultValue(String.valueOf(100))
+          .withDocumentation("In the first time to write, if the user 
estimated the size of the record and the actual deviation is large, and write a 
huge amount of data, "

Review Comment:
   Doc needs to be updated?



##########
hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecord.java:
##########
@@ -391,6 +392,11 @@ protected static boolean hasMetaFields(Schema schema) {
     return schema.getField(HoodieRecord.RECORD_KEY_METADATA_FIELD) != null;
   }
 
+  public long estimateSerializedSize() {
+    return ObjectSizeCalculator.getObjectSize(this);

Review Comment:
   I think it should be `data`, not the whole object?



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BaseSparkCommitActionExecutor.java:
##########
@@ -166,6 +168,12 @@ public HoodieWriteMetadata<HoodieData<WriteStatus>> 
execute(HoodieData<HoodieRec
       LOG.info("Input workload profile :" + workloadProfile);
     }
 
+    // first commit, need sample data from inputRDD to estimate average bytes 
per record.
+    if (config.getRecordSizeDynamicSamplingEnable() && 
table.getMetaClient().getActiveTimeline().getCommitTimeline().filterCompletedInstants().empty())
 {
+      int realSampleRecordAvgSize = dynamicSampleRecordSize(inputRDD);
+      config.setValue(COPY_ON_WRITE_RECORD_SIZE_ESTIMATE, 
String.valueOf(realSampleRecordAvgSize));

Review Comment:
   I think we can log here to users that `COPY_ON_WRITE_RECORD_SIZE_ESTIMATE` 
is updated by `Sample`



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to