yanghua commented on a change in pull request #2434: URL: https://github.com/apache/hudi/pull/2434#discussion_r556194645
########## File path: hudi-flink/src/main/java/org/apache/hudi/operator/InstantGenerateOperator.java ########## @@ -102,65 +106,73 @@ 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 + createinstantGenerateTmpFolder(); + } } @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 instantGenerateInfoFileName = String.format("%d_%d_%d",indexOfThisSubtask,checkpointId,batchSize); Review comment: Many lines have the same issue, please change the code style like this: `xxx,xxx` -> `xxx, xxx`. ########## 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) { + LOG.warn("waiting generate file, checkpointId [{}]",checkpointId); + tryTimes = 0; + } + } + + boolean hasData = false; + // judge whether has data in this checkpoint and delete tmp file. + for (FileStatus fileStatus : fileStatuses) { + Path path = fileStatus.getPath(); + String name = path.getName(); + // has data + if (Long.parseLong(name.split(UNDERLINE)[2]) > 0) { + hasData = true; + break; + } + } + + // delete all tmp file + fileStatuses = fs.listStatus(generatePath); + for (FileStatus fileStatus : fileStatuses) { + fs.delete(fileStatus.getPath()); + } + + return hasData; + } + + private void createinstantGenerateTmpFolder() throws IOException { Review comment: -> `createInstantGenerateTmpDir`? ########## 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) { + LOG.warn("waiting generate file, checkpointId [{}]",checkpointId); + tryTimes = 0; + } + } + + boolean hasData = false; Review comment: -> `receivedData`? ########## File path: hudi-flink/src/main/java/org/apache/hudi/operator/InstantGenerateOperator.java ########## @@ -71,22 +72,25 @@ 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_GENERATE_FOLDER_NAME = "instant_generate_tmp"; + private transient boolean isMain = false; + private transient volatile long batchSize = 0L; @Override public void processElement(StreamRecord<HoodieRecord> streamRecord) throws Exception { if (streamRecord.getValue() != null) { - bufferedRecords.add(streamRecord); output.collect(streamRecord); + batchSize++; } } @Override public void open() throws Exception { super.open(); + isMain = getRuntimeContext().getIndexOfThisSubtask() == 0; Review comment: Wet call `getRuntimeContext().getIndexOfThisSubtask()` several times, can we define a variable to store it? ########## File path: hudi-flink/src/main/java/org/apache/hudi/operator/InstantGenerateOperator.java ########## @@ -102,65 +106,73 @@ 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 + createinstantGenerateTmpFolder(); + } } @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 instantGenerateInfoFileName = String.format("%d_%d_%d",indexOfThisSubtask,checkpointId,batchSize); + Path path = new Path(INSTANT_GENERATE_FOLDER_NAME,instantGenerateInfoFileName); + // mk generate file by each subtask + fs.create(path,true); + LOG.info("subtask [{}] at checkpoint [{}] created generate file [{}]",indexOfThisSubtask,checkpointId,instantGenerateInfoFileName); + 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<String>("latestInstant", String.class); + latestInstantState = context.getOperatorStateStore().getListState(latestInstantStateDescriptor); + + if (context.isRestored()) { + Iterator<String> latestInstantIterator = latestInstantState.get().iterator(); + latestInstantIterator.forEachRemaining(x -> latestInstant = x); + LOG.info("InstantGenerateOperator initializeState get latestInstant [{}]", latestInstant); Review comment: ` Restoring the latest instant [{}] from the state.` sounds better? ---------------------------------------------------------------- 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