yanghua commented on a change in pull request #2176: URL: https://github.com/apache/hudi/pull/2176#discussion_r518606142
########## File path: hudi-flink-writer/src/main/java/org/apache/hudi/HudiFlinkStreamer.java ########## @@ -0,0 +1,164 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi; + +import com.beust.jcommander.IStringConverter; +import com.beust.jcommander.JCommander; +import com.beust.jcommander.Parameter; +import com.beust.jcommander.ParameterException; +import org.apache.flink.api.common.serialization.SimpleStringSchema; +import org.apache.flink.api.common.typeinfo.TypeHint; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.state.filesystem.FsStateBackend; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer; +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload; +import org.apache.hudi.constant.Operation; +import org.apache.hudi.operator.InstantGenerateOperator; +import org.apache.hudi.operator.KeyedWriteProcessFunction; +import org.apache.hudi.operator.WriteProcessOperator; +import org.apache.hudi.sink.CommitSink; +import org.apache.hudi.source.KafkaJson2HoodieRecord; +import org.apache.hudi.util.StreamerUtil; + +import java.util.ArrayList; +import java.util.List; +import java.util.Properties; + +public class HudiFlinkStreamer { + public static void main(String[] args) throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + final Config cfg = new Config(); + JCommander cmd = new JCommander(cfg, null, args); + if (cfg.help || args.length == 0) { + cmd.usage(); + System.exit(1); + } + env.enableCheckpointing(cfg.checkpointInterval); + env.getConfig().setGlobalJobParameters(cfg); + env.getCheckpointConfig().setMaxConcurrentCheckpoints(1); + env.disableOperatorChaining(); + + if (cfg.flinkCheckPointPath != null) { + env.setStateBackend(new FsStateBackend(cfg.flinkCheckPointPath)); + } + + Properties kafkaProps = StreamerUtil.getKafkaProps(cfg); + + // read from kafka source + DataStream<HoodieRecord> inputRecords = + env.addSource(new FlinkKafkaConsumer<>(cfg.kafkaTopic, new SimpleStringSchema(), kafkaProps)) + .map(new KafkaJson2HoodieRecord(cfg)) + .name("kafka_to_hudi_record") + .uid("kafka_to_hudi_record_uid"); + + inputRecords.transform(InstantGenerateOperator.NAME, TypeInformation.of(HoodieRecord.class), new InstantGenerateOperator()) + .setParallelism(1) + .keyBy(HoodieRecord::getPartitionPath) + .transform(WriteProcessOperator.NAME, TypeInformation.of(new TypeHint<Tuple3<String, List<WriteStatus>, Integer>>() { + }), new WriteProcessOperator(new KeyedWriteProcessFunction())).name("write_process").uid("write_process_uid") Review comment: The indent is bad. ########## File path: hudi-flink-writer/src/main/java/org/apache/hudi/HudiFlinkStreamer.java ########## @@ -0,0 +1,164 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi; + +import com.beust.jcommander.IStringConverter; +import com.beust.jcommander.JCommander; +import com.beust.jcommander.Parameter; +import com.beust.jcommander.ParameterException; +import org.apache.flink.api.common.serialization.SimpleStringSchema; +import org.apache.flink.api.common.typeinfo.TypeHint; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.state.filesystem.FsStateBackend; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer; +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload; +import org.apache.hudi.constant.Operation; +import org.apache.hudi.operator.InstantGenerateOperator; +import org.apache.hudi.operator.KeyedWriteProcessFunction; +import org.apache.hudi.operator.WriteProcessOperator; +import org.apache.hudi.sink.CommitSink; +import org.apache.hudi.source.KafkaJson2HoodieRecord; +import org.apache.hudi.util.StreamerUtil; + +import java.util.ArrayList; +import java.util.List; +import java.util.Properties; + +public class HudiFlinkStreamer { + public static void main(String[] args) throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + final Config cfg = new Config(); + JCommander cmd = new JCommander(cfg, null, args); + if (cfg.help || args.length == 0) { + cmd.usage(); + System.exit(1); + } + env.enableCheckpointing(cfg.checkpointInterval); + env.getConfig().setGlobalJobParameters(cfg); + env.getCheckpointConfig().setMaxConcurrentCheckpoints(1); + env.disableOperatorChaining(); + + if (cfg.flinkCheckPointPath != null) { + env.setStateBackend(new FsStateBackend(cfg.flinkCheckPointPath)); + } + + Properties kafkaProps = StreamerUtil.getKafkaProps(cfg); + + // read from kafka source + DataStream<HoodieRecord> inputRecords = + env.addSource(new FlinkKafkaConsumer<>(cfg.kafkaTopic, new SimpleStringSchema(), kafkaProps)) + .map(new KafkaJson2HoodieRecord(cfg)) + .name("kafka_to_hudi_record") + .uid("kafka_to_hudi_record_uid"); + + inputRecords.transform(InstantGenerateOperator.NAME, TypeInformation.of(HoodieRecord.class), new InstantGenerateOperator()) + .setParallelism(1) + .keyBy(HoodieRecord::getPartitionPath) + .transform(WriteProcessOperator.NAME, TypeInformation.of(new TypeHint<Tuple3<String, List<WriteStatus>, Integer>>() { + }), new WriteProcessOperator(new KeyedWriteProcessFunction())).name("write_process").uid("write_process_uid") + .setParallelism(env.getParallelism()) + .addSink(new CommitSink()).name("commit_sink").uid("commit_sink_uid") + .setParallelism(1); + + env.execute("Hudi write via Flink"); Review comment: `Hudi write job via Flink`? And can we add some variables to distinguish different hudi job? e.g. table name? ########## File path: hudi-flink-writer/src/main/java/org/apache/hudi/operator/InstantGenerateOperator.java ########## @@ -0,0 +1,208 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.operator; + +import org.apache.flink.api.common.state.ListState; +import org.apache.flink.api.common.state.ListStateDescriptor; +import org.apache.flink.runtime.state.StateInitializationContext; +import org.apache.flink.runtime.state.StateSnapshotContext; +import org.apache.flink.streaming.api.operators.AbstractStreamOperator; +import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hudi.HudiFlinkStreamer; +import org.apache.hudi.client.FlinkTaskContextSupplier; +import org.apache.hudi.client.HoodieFlinkWriteClient; +import org.apache.hudi.client.common.HoodieFlinkEngineContext; +import org.apache.hudi.client.common.TaskContextSupplier; +import org.apache.hudi.common.config.SerializableConfiguration; +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieTableType; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.util.StreamerUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.LinkedList; +import java.util.List; + +public class InstantGenerateOperator extends AbstractStreamOperator<HoodieRecord> implements OneInputStreamOperator<HoodieRecord, HoodieRecord> { + + private static final Logger LOG = LoggerFactory.getLogger(InstantGenerateOperator.class); + public static final String NAME = "InstantGenerateOperator"; + private HudiFlinkStreamer.Config cfg; + private HoodieWriteConfig writeConfig; + private HoodieFlinkWriteClient writeClient; + private SerializableConfiguration serializableHadoopConf; + private transient FileSystem fs; + private String latestInstant = ""; + List<String> latestInstantList = new ArrayList<>(1); + private transient ListState<String> latestInstantState; + private List<StreamRecord> records = new LinkedList(); Review comment: It seems `bufferedRecords` sounds better? ########## File path: hudi-flink-writer/src/main/java/org/apache/hudi/HudiFlinkStreamer.java ########## @@ -0,0 +1,164 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi; + +import com.beust.jcommander.IStringConverter; +import com.beust.jcommander.JCommander; +import com.beust.jcommander.Parameter; +import com.beust.jcommander.ParameterException; +import org.apache.flink.api.common.serialization.SimpleStringSchema; +import org.apache.flink.api.common.typeinfo.TypeHint; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.state.filesystem.FsStateBackend; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer; +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload; +import org.apache.hudi.constant.Operation; +import org.apache.hudi.operator.InstantGenerateOperator; +import org.apache.hudi.operator.KeyedWriteProcessFunction; +import org.apache.hudi.operator.WriteProcessOperator; +import org.apache.hudi.sink.CommitSink; +import org.apache.hudi.source.KafkaJson2HoodieRecord; +import org.apache.hudi.util.StreamerUtil; + +import java.util.ArrayList; +import java.util.List; +import java.util.Properties; + +public class HudiFlinkStreamer { + public static void main(String[] args) throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + final Config cfg = new Config(); + JCommander cmd = new JCommander(cfg, null, args); + if (cfg.help || args.length == 0) { + cmd.usage(); + System.exit(1); + } + env.enableCheckpointing(cfg.checkpointInterval); + env.getConfig().setGlobalJobParameters(cfg); + env.getCheckpointConfig().setMaxConcurrentCheckpoints(1); + env.disableOperatorChaining(); + + if (cfg.flinkCheckPointPath != null) { + env.setStateBackend(new FsStateBackend(cfg.flinkCheckPointPath)); + } + + Properties kafkaProps = StreamerUtil.getKafkaProps(cfg); + + // read from kafka source + DataStream<HoodieRecord> inputRecords = + env.addSource(new FlinkKafkaConsumer<>(cfg.kafkaTopic, new SimpleStringSchema(), kafkaProps)) + .map(new KafkaJson2HoodieRecord(cfg)) + .name("kafka_to_hudi_record") + .uid("kafka_to_hudi_record_uid"); + + inputRecords.transform(InstantGenerateOperator.NAME, TypeInformation.of(HoodieRecord.class), new InstantGenerateOperator()) + .setParallelism(1) Review comment: Add a description about why hard code this parallelism. ########## File path: hudi-flink-writer/src/main/java/org/apache/hudi/operator/InstantGenerateOperator.java ########## @@ -0,0 +1,208 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.operator; + +import org.apache.flink.api.common.state.ListState; +import org.apache.flink.api.common.state.ListStateDescriptor; +import org.apache.flink.runtime.state.StateInitializationContext; +import org.apache.flink.runtime.state.StateSnapshotContext; +import org.apache.flink.streaming.api.operators.AbstractStreamOperator; +import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hudi.HudiFlinkStreamer; +import org.apache.hudi.client.FlinkTaskContextSupplier; +import org.apache.hudi.client.HoodieFlinkWriteClient; +import org.apache.hudi.client.common.HoodieFlinkEngineContext; +import org.apache.hudi.client.common.TaskContextSupplier; +import org.apache.hudi.common.config.SerializableConfiguration; +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieTableType; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.util.StreamerUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.LinkedList; +import java.util.List; + +public class InstantGenerateOperator extends AbstractStreamOperator<HoodieRecord> implements OneInputStreamOperator<HoodieRecord, HoodieRecord> { + + private static final Logger LOG = LoggerFactory.getLogger(InstantGenerateOperator.class); + public static final String NAME = "InstantGenerateOperator"; + private HudiFlinkStreamer.Config cfg; + private HoodieWriteConfig writeConfig; + private HoodieFlinkWriteClient writeClient; + private SerializableConfiguration serializableHadoopConf; + private transient FileSystem fs; + private String latestInstant = ""; + List<String> latestInstantList = new ArrayList<>(1); Review comment: keep the same style as others? add the access modifier ########## File path: hudi-flink-writer/src/main/java/org/apache/hudi/operator/InstantGenerateOperator.java ########## @@ -0,0 +1,208 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.operator; + +import org.apache.flink.api.common.state.ListState; +import org.apache.flink.api.common.state.ListStateDescriptor; +import org.apache.flink.runtime.state.StateInitializationContext; +import org.apache.flink.runtime.state.StateSnapshotContext; +import org.apache.flink.streaming.api.operators.AbstractStreamOperator; +import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hudi.HudiFlinkStreamer; +import org.apache.hudi.client.FlinkTaskContextSupplier; +import org.apache.hudi.client.HoodieFlinkWriteClient; +import org.apache.hudi.client.common.HoodieFlinkEngineContext; +import org.apache.hudi.client.common.TaskContextSupplier; +import org.apache.hudi.common.config.SerializableConfiguration; +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieTableType; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.util.StreamerUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.LinkedList; +import java.util.List; + +public class InstantGenerateOperator extends AbstractStreamOperator<HoodieRecord> implements OneInputStreamOperator<HoodieRecord, HoodieRecord> { + + private static final Logger LOG = LoggerFactory.getLogger(InstantGenerateOperator.class); + public static final String NAME = "InstantGenerateOperator"; Review comment: split the static fields and non-static fields ########## File path: hudi-flink-writer/src/main/java/org/apache/hudi/operator/InstantGenerateOperator.java ########## @@ -0,0 +1,208 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.operator; + +import org.apache.flink.api.common.state.ListState; +import org.apache.flink.api.common.state.ListStateDescriptor; +import org.apache.flink.runtime.state.StateInitializationContext; +import org.apache.flink.runtime.state.StateSnapshotContext; +import org.apache.flink.streaming.api.operators.AbstractStreamOperator; +import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hudi.HudiFlinkStreamer; +import org.apache.hudi.client.FlinkTaskContextSupplier; +import org.apache.hudi.client.HoodieFlinkWriteClient; +import org.apache.hudi.client.common.HoodieFlinkEngineContext; +import org.apache.hudi.client.common.TaskContextSupplier; +import org.apache.hudi.common.config.SerializableConfiguration; +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieTableType; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.util.StreamerUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.LinkedList; +import java.util.List; + +public class InstantGenerateOperator extends AbstractStreamOperator<HoodieRecord> implements OneInputStreamOperator<HoodieRecord, HoodieRecord> { + + private static final Logger LOG = LoggerFactory.getLogger(InstantGenerateOperator.class); + public static final String NAME = "InstantGenerateOperator"; + private HudiFlinkStreamer.Config cfg; + private HoodieWriteConfig writeConfig; + private HoodieFlinkWriteClient writeClient; + private SerializableConfiguration serializableHadoopConf; + private transient FileSystem fs; + private String latestInstant = ""; + List<String> latestInstantList = new ArrayList<>(1); + private transient ListState<String> latestInstantState; + private List<StreamRecord> records = new LinkedList(); + private transient ListState<StreamRecord> recordsState; + private Integer commitTimeout; + + @Override + public void processElement(StreamRecord<HoodieRecord> streamRecord) throws Exception { + if (streamRecord.getValue() != null) { + records.add(streamRecord); + output.collect(streamRecord); + } + } + + @Override + public void open() throws Exception { + super.open(); + // get configs from runtimeContext + cfg = (HudiFlinkStreamer.Config) getRuntimeContext().getExecutionConfig().getGlobalJobParameters(); + + // timeout + commitTimeout = Integer.valueOf(cfg.flinkCommitTimeout); + + // hadoopConf + serializableHadoopConf = new SerializableConfiguration(StreamerUtil.getHadoopConf()); + + // Hadoop FileSystem + fs = FSUtils.getFs(cfg.targetBasePath, serializableHadoopConf.get()); + + // HoodieWriteConfig + writeConfig = StreamerUtil.getHoodieClientConfig(cfg); + + TaskContextSupplier taskContextSupplier = new FlinkTaskContextSupplier(null); + + // writeClient + writeClient = new HoodieFlinkWriteClient(new HoodieFlinkEngineContext(taskContextSupplier), writeConfig); + + // init table, create it if not exists. + initTable(); + } + + @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)) { + doChecker(); + // last instant completed, set it empty + latestInstant = ""; + } + + // no data no new instant + if (!records.isEmpty()) { + latestInstant = startNewInstant(checkpointId); + } + super.prepareSnapshotPreBarrier(checkpointId); Review comment: Why do two times call this method? ---------------------------------------------------------------- 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]
