yanghua commented on a change in pull request #2434:
URL: https://github.com/apache/hudi/pull/2434#discussion_r557462779



##########
File path: 
hudi-flink/src/main/java/org/apache/hudi/operator/InstantGenerateOperator.java
##########
@@ -71,16 +73,18 @@
   private String latestInstant = "";
   private List<String> latestInstantList = new ArrayList<>(1);
   private transient ListState<String> latestInstantState;
-  private List<StreamRecord> bufferedRecords = new LinkedList();
-  private transient ListState<StreamRecord> recordsState;
   private Integer retryTimes;
   private Integer retryInterval;
+  private static final String UNDERLINE = "_";
+  private static final String INSTANT_MARKER_FOLDER_NAME = ".instant_marker";
+  private transient boolean isMain = false;
+  private transient AtomicLong batchSize = new AtomicLong(0);

Review comment:
       I still think `cpRecordCounter` sounds better.

##########
File path: 
hudi-flink/src/main/java/org/apache/hudi/operator/InstantGenerateOperator.java
##########
@@ -102,65 +106,76 @@ public void open() throws Exception {
     // Hadoop FileSystem
     fs = FSUtils.getFs(cfg.targetBasePath, serializableHadoopConf.get());
 
-    TaskContextSupplier taskContextSupplier = new 
FlinkTaskContextSupplier(null);
+    if (isMain) {
+      TaskContextSupplier taskContextSupplier = new 
FlinkTaskContextSupplier(null);
 
-    // writeClient
-    writeClient = new HoodieFlinkWriteClient(new 
HoodieFlinkEngineContext(taskContextSupplier), 
StreamerUtil.getHoodieClientConfig(cfg), true);
+      // writeClient
+      writeClient = new HoodieFlinkWriteClient(new 
HoodieFlinkEngineContext(taskContextSupplier), 
StreamerUtil.getHoodieClientConfig(cfg), true);
 
-    // init table, create it if not exists.
-    initTable();
+      // init table, create it if not exists.
+      initTable();
+
+      // create instant marker directory
+      createInstantMarkerDir();
+    }
   }
 
   @Override
   public void prepareSnapshotPreBarrier(long checkpointId) throws Exception {
     super.prepareSnapshotPreBarrier(checkpointId);
-    // check whether the last instant is completed, if not, wait 10s and then 
throws an exception
-    if (!StringUtils.isNullOrEmpty(latestInstant)) {
-      doCheck();
-      // last instant completed, set it empty
-      latestInstant = "";
-    }
+    int indexOfThisSubtask = getRuntimeContext().getIndexOfThisSubtask();
+    String instantMarkerFileName = String.format("%d_%d_%d", 
indexOfThisSubtask, checkpointId, batchSize.get());
+    Path path = new Path(new Path(HoodieTableMetaClient.AUXILIARYFOLDER_NAME, 
INSTANT_MARKER_FOLDER_NAME), instantMarkerFileName);

Review comment:
       Using `Paths.get(string, string, ...)` looks better?

##########
File path: 
hudi-flink/src/main/java/org/apache/hudi/operator/InstantGenerateOperator.java
##########
@@ -102,65 +106,76 @@ public void open() throws Exception {
     // Hadoop FileSystem
     fs = FSUtils.getFs(cfg.targetBasePath, serializableHadoopConf.get());
 
-    TaskContextSupplier taskContextSupplier = new 
FlinkTaskContextSupplier(null);
+    if (isMain) {
+      TaskContextSupplier taskContextSupplier = new 
FlinkTaskContextSupplier(null);
 
-    // writeClient
-    writeClient = new HoodieFlinkWriteClient(new 
HoodieFlinkEngineContext(taskContextSupplier), 
StreamerUtil.getHoodieClientConfig(cfg), true);
+      // writeClient
+      writeClient = new HoodieFlinkWriteClient(new 
HoodieFlinkEngineContext(taskContextSupplier), 
StreamerUtil.getHoodieClientConfig(cfg), true);
 
-    // init table, create it if not exists.
-    initTable();
+      // init table, create it if not exists.
+      initTable();
+
+      // create instant marker directory
+      createInstantMarkerDir();
+    }
   }
 
   @Override
   public void prepareSnapshotPreBarrier(long checkpointId) throws Exception {
     super.prepareSnapshotPreBarrier(checkpointId);
-    // check whether the last instant is completed, if not, wait 10s and then 
throws an exception
-    if (!StringUtils.isNullOrEmpty(latestInstant)) {
-      doCheck();
-      // last instant completed, set it empty
-      latestInstant = "";
-    }
+    int indexOfThisSubtask = getRuntimeContext().getIndexOfThisSubtask();
+    String instantMarkerFileName = String.format("%d_%d_%d", 
indexOfThisSubtask, checkpointId, batchSize.get());
+    Path path = new Path(new Path(HoodieTableMetaClient.AUXILIARYFOLDER_NAME, 
INSTANT_MARKER_FOLDER_NAME), instantMarkerFileName);
+    // mk generate file by each subtask
+    fs.create(path, true);
+    LOG.info("Subtask [{}] at checkpoint [{}] created generate file [{}]", 
indexOfThisSubtask, checkpointId, instantMarkerFileName);
+    if (isMain) {
+      boolean receivedDataInCurrentCP = checkReceivedData(checkpointId);
+      // check whether the last instant is completed, if not, wait 10s and 
then throws an exception
+      if (!StringUtils.isNullOrEmpty(latestInstant)) {
+        doCheck();
+        // last instant completed, set it empty
+        latestInstant = "";
+      }
 
-    // no data no new instant
-    if (!bufferedRecords.isEmpty()) {
-      latestInstant = startNewInstant(checkpointId);
+      // no data no new instant
+      if (receivedDataInCurrentCP) {
+        latestInstant = startNewInstant(checkpointId);
+      }
     }
   }
 
   @Override
   public void initializeState(StateInitializationContext context) throws 
Exception {
-    // instantState
-    ListStateDescriptor<String> latestInstantStateDescriptor = new 
ListStateDescriptor<String>("latestInstant", String.class);
-    latestInstantState = 
context.getOperatorStateStore().getListState(latestInstantStateDescriptor);
-
-    // recordState
-    ListStateDescriptor<StreamRecord> recordsStateDescriptor = new 
ListStateDescriptor<StreamRecord>("recordsState", StreamRecord.class);
-    recordsState = 
context.getOperatorStateStore().getListState(recordsStateDescriptor);
-
-    if (context.isRestored()) {
-      Iterator<String> latestInstantIterator = 
latestInstantState.get().iterator();
-      latestInstantIterator.forEachRemaining(x -> latestInstant = x);
-      LOG.info("InstantGenerateOperator initializeState get latestInstant 
[{}]", latestInstant);
-
-      Iterator<StreamRecord> recordIterator = recordsState.get().iterator();
-      bufferedRecords.clear();
-      recordIterator.forEachRemaining(x -> bufferedRecords.add(x));
+    isMain = getRuntimeContext().getIndexOfThisSubtask() == 0;
+    if (isMain) {
+      // instantState
+      ListStateDescriptor<String> latestInstantStateDescriptor = new 
ListStateDescriptor<>("latestInstant", String.class);
+      latestInstantState = 
context.getOperatorStateStore().getListState(latestInstantStateDescriptor);
+
+      if (context.isRestored()) {
+        Iterator<String> latestInstantIterator = 
latestInstantState.get().iterator();
+        latestInstantIterator.forEachRemaining(x -> latestInstant = x);
+        LOG.info("Restoring the latest instant [{}] from the state", 
latestInstant);
+      }
     }
   }
 
   @Override
   public void snapshotState(StateSnapshotContext functionSnapshotContext) 
throws Exception {
-    if (latestInstantList.isEmpty()) {
-      latestInstantList.add(latestInstant);
+    long checkpointId = functionSnapshotContext.getCheckpointId();
+    if (isMain) {
+      LOG.info("Update latest instant [{}] records size [{}] checkpointId 
[{}]", latestInstant, batchSize, checkpointId);
+      if (latestInstantList.isEmpty()) {
+        latestInstantList.add(latestInstant);
+      } else {
+        latestInstantList.set(0, latestInstant);
+      }
+      latestInstantState.update(latestInstantList);
     } else {
-      latestInstantList.set(0, latestInstant);
+      LOG.info("Records size [{}] checkpointId [{}]", batchSize, checkpointId);

Review comment:
       `Task instance %d received %d records in checkpoint [%d]` looks better?

##########
File path: 
hudi-flink/src/main/java/org/apache/hudi/operator/InstantGenerateOperator.java
##########
@@ -102,65 +105,76 @@ public void open() throws Exception {
     // Hadoop FileSystem
     fs = FSUtils.getFs(cfg.targetBasePath, serializableHadoopConf.get());
 
-    TaskContextSupplier taskContextSupplier = new 
FlinkTaskContextSupplier(null);
+    if (isMain) {
+      TaskContextSupplier taskContextSupplier = new 
FlinkTaskContextSupplier(null);

Review comment:
       wdyt about this review suggestion?

##########
File path: 
hudi-flink/src/main/java/org/apache/hudi/operator/InstantGenerateOperator.java
##########
@@ -222,4 +236,59 @@ public void close() throws Exception {
       fs.close();
     }
   }
+
+  private boolean checkReceivedData(long checkpointId) throws 
InterruptedException, IOException {
+    int numberOfParallelSubtasks = 
getRuntimeContext().getNumberOfParallelSubtasks();
+    FileStatus[] fileStatuses = null;
+    Path generatePath = new Path(INSTANT_GENERATE_FOLDER_NAME);
+    int tryTimes = 1;
+    // waiting all subtask create generate file ready
+    while (true) {
+      Thread.sleep(500L);
+      fileStatuses = fs.listStatus(generatePath, new PathFilter() {
+        @Override
+        public boolean accept(Path pathname) {
+          return pathname.getName().contains(String.format("_%d_", 
checkpointId));

Review comment:
       Can this suggestion be accepted?

##########
File path: 
hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java
##########
@@ -202,7 +203,7 @@ public String getTempFolderPath() {
 
   /**
    * Returns Marker folder path.
-   * 

Review comment:
       Still exist, you can add a whitespace after `*`, it may solve the issue.

##########
File path: 
hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java
##########
@@ -272,7 +273,7 @@ public HoodieWrapperFileSystem getFs() {
 
   /**
    * Return raw file-system.
-   * 

Review comment:
       Or just revert this file's change via git command.

##########
File path: 
hudi-flink/src/main/java/org/apache/hudi/operator/InstantGenerateOperator.java
##########
@@ -222,4 +237,60 @@ public void close() throws Exception {
       fs.close();
     }
   }
+
+  private boolean checkReceivedData(long checkpointId) throws 
InterruptedException, IOException {
+    int numberOfParallelSubtasks = 
getRuntimeContext().getNumberOfParallelSubtasks();
+    FileStatus[] fileStatuses;
+    Path instantMarkerPath = new 
Path(HoodieTableMetaClient.AUXILIARYFOLDER_NAME, INSTANT_MARKER_FOLDER_NAME);
+    int tryTimes = 1;
+    // waiting all subtask create marker file ready
+    while (true) {
+      Thread.sleep(500L);
+      fileStatuses = fs.listStatus(instantMarkerPath, new PathFilter() {
+        @Override
+        public boolean accept(Path pathname) {
+          return pathname.getName().contains(String.format("_%d_", 
checkpointId));
+        }
+      });
+
+      // is ready
+      if (fileStatuses != null && fileStatuses.length == 
numberOfParallelSubtasks) {
+        break;
+      }
+
+      if (tryTimes >= 5) {
+        LOG.warn("waiting generate file, checkpointId [{}]", checkpointId);
+        tryTimes = 0;
+      }
+      tryTimes++;
+    }
+
+    boolean receivedData = false;
+    // judge whether has data in this checkpoint and delete maker file.
+    for (FileStatus fileStatus : fileStatuses) {
+      Path path = fileStatus.getPath();
+      String name = path.getName();
+      // has data
+      if (Long.parseLong(name.split(UNDERLINE)[2]) > 0) {
+        receivedData = true;
+        break;
+      }
+    }
+
+    // delete all marker file
+    fileStatuses = fs.listStatus(instantMarkerPath);
+    for (FileStatus fileStatus : fileStatuses) {
+      fs.delete(fileStatus.getPath(), true);
+    }
+
+    return receivedData;
+  }
+
+  private void createInstantMarkerDir() throws IOException {
+    // Always create instantMarkerFolder which is needed for 
InstantGenerateOperator
+    final Path instantMarkerFolder = new Path(new Path(cfg.targetBasePath, 
HoodieTableMetaClient.AUXILIARYFOLDER_NAME), INSTANT_MARKER_FOLDER_NAME);

Review comment:
       `Paths.get()`

##########
File path: 
hudi-flink/src/main/java/org/apache/hudi/operator/InstantGenerateOperator.java
##########
@@ -222,4 +234,59 @@ public void close() throws Exception {
       fs.close();
     }
   }
+
+  private boolean checkReceivedData(long checkpointId) throws 
InterruptedException, IOException {
+    int numberOfParallelSubtasks = 
getRuntimeContext().getNumberOfParallelSubtasks();
+    FileStatus[] fileStatuses = null;
+    Path generatePath = new Path(INSTANT_GENERATE_FOLDER_NAME);
+    int tryTimes = 1;
+    // waiting all subtask create generate file ready
+    while (true) {
+      Thread.sleep(500L);
+      fileStatuses = fs.listStatus(generatePath, new PathFilter() {
+        @Override
+        public boolean accept(Path pathname) {
+          return 
pathname.getName().contains(String.format("_%d_",checkpointId));
+        }
+      });
+
+      // is ready
+      if (fileStatuses != null && fileStatuses.length == 
numberOfParallelSubtasks) {
+        break;
+      }
+
+      if (tryTimes >= 5) {

Review comment:
       ?




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

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


Reply via email to