yanghua commented on a change in pull request #2434: URL: https://github.com/apache/hudi/pull/2434#discussion_r556975899
########## 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: IMO, here we can set the argument of FlinkTaskContextSupplier to the instance of `RuntimeContext`, right? ########## 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); - // 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 instantGenerateTmpFolder Review comment: About the comment, we'd better avoid copying the method name, `create temp folder for generating instant` seems better? ########## 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: You have defined the separator by `UNDERLINE`, it would be better to use that. wdyt? ########## 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: Did you check the logic, how to make the `tryTimes` up to 5? ---------------------------------------------------------------- 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: us...@infra.apache.org