nsivabalan commented on code in PR #5470:
URL: https://github.com/apache/hudi/pull/5470#discussion_r869662469


##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowCreateHandle.java:
##########
@@ -39,49 +39,77 @@
 import org.apache.log4j.Logger;
 import org.apache.spark.sql.catalyst.InternalRow;
 import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
 
 import java.io.IOException;
 import java.io.Serializable;
 import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Function;
 
 /**
  * Create handle with InternalRow for datasource implementation of bulk insert.
  */
 public class HoodieRowCreateHandle implements Serializable {
 
   private static final long serialVersionUID = 1L;
+
   private static final Logger LOG = 
LogManager.getLogger(HoodieRowCreateHandle.class);
-  private static final AtomicLong SEQGEN = new AtomicLong(1);
+  private static final AtomicLong GLOBAL_SEQ_NO = new AtomicLong(1);
+
+  private static final Integer RECORD_KEY_META_FIELD_ORD =
+      
HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS.get(HoodieRecord.RECORD_KEY_METADATA_FIELD);
+  private static final Integer PARTITION_PATH_META_FIELD_ORD =
+      
HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS.get(HoodieRecord.PARTITION_PATH_METADATA_FIELD);
 
-  private final String instantTime;
-  private final int taskPartitionId;
-  private final long taskId;
-  private final long taskEpochId;
   private final HoodieTable table;
   private final HoodieWriteConfig writeConfig;
-  protected final HoodieInternalRowFileWriter fileWriter;
+
+  private final FileSystem fs;
+
   private final String partitionPath;
   private final Path path;
   private final String fileId;
-  private final FileSystem fs;
-  protected final HoodieInternalWriteStatus writeStatus;
+
+  private final boolean populateMetaFields;
+
+  private final UTF8String fileName;
+  private final UTF8String commitTime;
+  private final Function<Long, String> seqIdGenerator;
+
   private final HoodieTimer currTimer;
 
-  public HoodieRowCreateHandle(HoodieTable table, HoodieWriteConfig 
writeConfig, String partitionPath, String fileId,
-      String instantTime, int taskPartitionId, long taskId, long taskEpochId,
-      StructType structType) {
+  protected final HoodieInternalRowFileWriter fileWriter;
+  protected final HoodieInternalWriteStatus writeStatus;
+
+  public HoodieRowCreateHandle(HoodieTable table,
+                               HoodieWriteConfig writeConfig,
+                               String partitionPath,
+                               String fileId,
+                               String instantTime,
+                               int taskPartitionId,
+                               long taskId,
+                               long taskEpochId,
+                               StructType structType,
+                               boolean populateMetaFields) {
     this.partitionPath = partitionPath;
     this.table = table;
     this.writeConfig = writeConfig;
-    this.instantTime = instantTime;
-    this.taskPartitionId = taskPartitionId;
-    this.taskId = taskId;
-    this.taskEpochId = taskEpochId;
     this.fileId = fileId;
-    this.currTimer = new HoodieTimer();
-    this.currTimer.startTimer();
+
+    this.currTimer = new HoodieTimer(true);
+
     this.fs = table.getMetaClient().getFs();
-    this.path = makeNewPath(partitionPath);
+
+    String writeToken = getWriteToken(taskPartitionId, taskId, taskEpochId);
+    String fileName = FSUtils.makeDataFileName(instantTime, writeToken, fileId,
+        
table.getMetaClient().getTableConfig().getBaseFileFormat().getFileExtension());

Review Comment:
   table.getBaseFileExtension()



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowCreateHandle.java:
##########
@@ -39,49 +39,77 @@
 import org.apache.log4j.Logger;
 import org.apache.spark.sql.catalyst.InternalRow;
 import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
 
 import java.io.IOException;
 import java.io.Serializable;
 import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Function;
 
 /**
  * Create handle with InternalRow for datasource implementation of bulk insert.
  */
 public class HoodieRowCreateHandle implements Serializable {
 
   private static final long serialVersionUID = 1L;
+
   private static final Logger LOG = 
LogManager.getLogger(HoodieRowCreateHandle.class);
-  private static final AtomicLong SEQGEN = new AtomicLong(1);
+  private static final AtomicLong GLOBAL_SEQ_NO = new AtomicLong(1);
+
+  private static final Integer RECORD_KEY_META_FIELD_ORD =
+      
HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS.get(HoodieRecord.RECORD_KEY_METADATA_FIELD);
+  private static final Integer PARTITION_PATH_META_FIELD_ORD =
+      
HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS.get(HoodieRecord.PARTITION_PATH_METADATA_FIELD);
 
-  private final String instantTime;
-  private final int taskPartitionId;
-  private final long taskId;
-  private final long taskEpochId;
   private final HoodieTable table;
   private final HoodieWriteConfig writeConfig;
-  protected final HoodieInternalRowFileWriter fileWriter;
+
+  private final FileSystem fs;
+
   private final String partitionPath;
   private final Path path;
   private final String fileId;
-  private final FileSystem fs;
-  protected final HoodieInternalWriteStatus writeStatus;
+
+  private final boolean populateMetaFields;
+
+  private final UTF8String fileName;
+  private final UTF8String commitTime;
+  private final Function<Long, String> seqIdGenerator;
+
   private final HoodieTimer currTimer;
 
-  public HoodieRowCreateHandle(HoodieTable table, HoodieWriteConfig 
writeConfig, String partitionPath, String fileId,
-      String instantTime, int taskPartitionId, long taskId, long taskEpochId,
-      StructType structType) {
+  protected final HoodieInternalRowFileWriter fileWriter;
+  protected final HoodieInternalWriteStatus writeStatus;
+
+  public HoodieRowCreateHandle(HoodieTable table,
+                               HoodieWriteConfig writeConfig,
+                               String partitionPath,
+                               String fileId,
+                               String instantTime,
+                               int taskPartitionId,
+                               long taskId,
+                               long taskEpochId,
+                               StructType structType,
+                               boolean populateMetaFields) {
     this.partitionPath = partitionPath;
     this.table = table;
     this.writeConfig = writeConfig;
-    this.instantTime = instantTime;
-    this.taskPartitionId = taskPartitionId;
-    this.taskId = taskId;
-    this.taskEpochId = taskEpochId;
     this.fileId = fileId;
-    this.currTimer = new HoodieTimer();
-    this.currTimer.startTimer();
+
+    this.currTimer = new HoodieTimer(true);
+

Review Comment:
   startTimer ?



-- 
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