danny0405 commented on a change in pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#discussion_r561768669



##########
File path: 
hudi-flink/src/main/java/org/apache/hudi/operator/StreamWriteFunction.java
##########
@@ -0,0 +1,344 @@
+/*
+ * 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.hudi.avro.HoodieAvroUtils;
+import org.apache.hudi.client.FlinkTaskContextSupplier;
+import org.apache.hudi.client.HoodieFlinkWriteClient;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.client.common.HoodieFlinkEngineContext;
+import org.apache.hudi.common.config.SerializableConfiguration;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.common.model.WriteOperationType;
+import org.apache.hudi.common.util.ObjectSizeCalculator;
+import org.apache.hudi.keygen.KeyGenerator;
+import org.apache.hudi.operator.event.BatchWriteSuccessEvent;
+import org.apache.hudi.util.StreamerUtil;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.formats.avro.RowDataToAvroConverters;
+import org.apache.flink.runtime.operators.coordination.OperatorEventGateway;
+import org.apache.flink.runtime.state.FunctionInitializationContext;
+import org.apache.flink.runtime.state.FunctionSnapshotContext;
+import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
+import org.apache.flink.streaming.api.functions.KeyedProcessFunction;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.util.Collector;
+import org.apache.flink.util.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Random;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.function.BiFunction;
+
+/**
+ * Sink function to write the data to the underneath filesystem.
+ *
+ * <p><h2>Work Flow</h2>
+ *
+ * <p>The function firstly buffers the data as a batch of {@link 
HoodieRecord}s,
+ * It flushes(write) the records batch when a Flink checkpoint starts. After a 
batch has been written successfully,
+ * the function notifies its operator coordinator {@link 
StreamWriteOperatorCoordinator} to mark a successful write.
+ *
+ * <p><h2>Exactly-once Semantics</h2>
+ *
+ * <p>The task implements exactly-once semantics by buffering the data between 
checkpoints. The operator coordinator
+ * starts a new instant on the time line when a checkpoint triggers, the 
coordinator checkpoints always
+ * start before its operator, so when this function starts a checkpoint, a 
REQUESTED instant already exists.
+ * The function process thread then block data buffering and the checkpoint 
thread starts flushing the existing data buffer.
+ * When the existing data buffer write successfully, the process thread 
unblock and start buffering again for the next round checkpoint.
+ * Because any checkpoint failures would trigger the write rollback, it 
implements the exactly-once semantics.
+ *
+ * <p><h2>Fault Tolerance</h2>
+ *
+ * <p>The operator coordinator checks the validity for the last instant when 
it starts a new one. The operator rolls back
+ * the written data and throws when any error occurs. This means any 
checkpoint or task failure would trigger a failover.
+ * The operator coordinator would try several times when committing the 
writestatus.
+ *
+ * <p>Note: The function task requires the input stream be partitioned by the 
partition fields to avoid different write tasks
+ * write to the same file group that conflict. The general case for partition 
path is a datetime field,
+ * so the sink task is very possible to have IO bottleneck, the more flexible 
solution is to shuffle the
+ * data by the file group IDs.
+ *
+ * @param <I> Type of the input record
+ * @see StreamWriteOperatorCoordinator
+ */
+public class StreamWriteFunction<K, I, O> extends KeyedProcessFunction<K, I, 
O> implements CheckpointedFunction {
+
+  private static final long serialVersionUID = 1L;
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(StreamWriteFunction.class);
+
+  /**
+   * Write buffer for a checkpoint.
+   */
+  private transient List<HoodieRecord> buffer;
+
+  /**
+   * The buffer lock to control data buffering/flushing.
+   */
+  private transient ReentrantLock bufferLock;
+
+  /**
+   * The condition to decide whether to add new records into the buffer.
+   */
+  private transient Condition addToBufferCondition;
+
+  /**
+   * Flag saying whether there is an on-going checkpoint.
+   */
+  private volatile boolean onCheckpointing = false;
+
+  /**
+   * Config options.
+   */
+  private final Configuration config;
+
+  /**
+   * Id of current subtask.
+   */
+  private int taskID;
+
+  /**
+   * Write Client.
+   */
+  private transient HoodieFlinkWriteClient writeClient;
+
+  private transient BiFunction<List<HoodieRecord>, String, List<WriteStatus>> 
writeFunction;
+
+  /**
+   * HoodieKey generator.
+   */
+  private transient KeyGenerator keyGenerator;
+
+  /**
+   * Row type of the input.
+   */
+  private final RowType rowType;
+
+  /**
+   * Avro schema of the input.
+   */
+  private final Schema avroSchema;
+
+  private transient RowDataToAvroConverters.RowDataToAvroConverter converter;
+
+  /**
+   * The REQUESTED instant we write the data.
+   */
+  private volatile String currentInstant;
+
+  /**
+   * Gateway to send operator events to the operator coordinator.
+   */
+  private transient OperatorEventGateway eventGateway;
+
+  /**
+   * Constructs a StreamingSinkFunction.
+   *
+   * @param rowType The input row type
+   * @param config  The config options
+   */
+  public StreamWriteFunction(RowType rowType, Configuration config) {

Review comment:
       Use row type has 2 benefits:
   1. Use the all kinds of `DeserializationSchema` provided for `RowData`, such 
as `JsonRowDataDeserializationSchema`, there is no need to write converter code 
for every data format, `RowData` is a good data structure to bridge external 
raw format with `HoodieRecord`
   2. `RowData` is internal data structure of `SQL` API, so that the codes for 
datastream can be reused.

##########
File path: 
hudi-flink/src/main/java/org/apache/hudi/operator/StreamWriteFunction.java
##########
@@ -0,0 +1,344 @@
+/*
+ * 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.hudi.avro.HoodieAvroUtils;
+import org.apache.hudi.client.FlinkTaskContextSupplier;
+import org.apache.hudi.client.HoodieFlinkWriteClient;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.client.common.HoodieFlinkEngineContext;
+import org.apache.hudi.common.config.SerializableConfiguration;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.common.model.WriteOperationType;
+import org.apache.hudi.common.util.ObjectSizeCalculator;
+import org.apache.hudi.keygen.KeyGenerator;
+import org.apache.hudi.operator.event.BatchWriteSuccessEvent;
+import org.apache.hudi.util.StreamerUtil;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.formats.avro.RowDataToAvroConverters;
+import org.apache.flink.runtime.operators.coordination.OperatorEventGateway;
+import org.apache.flink.runtime.state.FunctionInitializationContext;
+import org.apache.flink.runtime.state.FunctionSnapshotContext;
+import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
+import org.apache.flink.streaming.api.functions.KeyedProcessFunction;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.util.Collector;
+import org.apache.flink.util.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Random;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.function.BiFunction;
+
+/**
+ * Sink function to write the data to the underneath filesystem.
+ *
+ * <p><h2>Work Flow</h2>
+ *
+ * <p>The function firstly buffers the data as a batch of {@link 
HoodieRecord}s,
+ * It flushes(write) the records batch when a Flink checkpoint starts. After a 
batch has been written successfully,
+ * the function notifies its operator coordinator {@link 
StreamWriteOperatorCoordinator} to mark a successful write.
+ *
+ * <p><h2>Exactly-once Semantics</h2>
+ *
+ * <p>The task implements exactly-once semantics by buffering the data between 
checkpoints. The operator coordinator
+ * starts a new instant on the time line when a checkpoint triggers, the 
coordinator checkpoints always
+ * start before its operator, so when this function starts a checkpoint, a 
REQUESTED instant already exists.
+ * The function process thread then block data buffering and the checkpoint 
thread starts flushing the existing data buffer.
+ * When the existing data buffer write successfully, the process thread 
unblock and start buffering again for the next round checkpoint.
+ * Because any checkpoint failures would trigger the write rollback, it 
implements the exactly-once semantics.
+ *
+ * <p><h2>Fault Tolerance</h2>
+ *
+ * <p>The operator coordinator checks the validity for the last instant when 
it starts a new one. The operator rolls back
+ * the written data and throws when any error occurs. This means any 
checkpoint or task failure would trigger a failover.
+ * The operator coordinator would try several times when committing the 
writestatus.
+ *
+ * <p>Note: The function task requires the input stream be partitioned by the 
partition fields to avoid different write tasks
+ * write to the same file group that conflict. The general case for partition 
path is a datetime field,
+ * so the sink task is very possible to have IO bottleneck, the more flexible 
solution is to shuffle the
+ * data by the file group IDs.
+ *
+ * @param <I> Type of the input record
+ * @see StreamWriteOperatorCoordinator
+ */
+public class StreamWriteFunction<K, I, O> extends KeyedProcessFunction<K, I, 
O> implements CheckpointedFunction {
+
+  private static final long serialVersionUID = 1L;
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(StreamWriteFunction.class);
+
+  /**
+   * Write buffer for a checkpoint.
+   */
+  private transient List<HoodieRecord> buffer;
+
+  /**
+   * The buffer lock to control data buffering/flushing.
+   */
+  private transient ReentrantLock bufferLock;
+
+  /**
+   * The condition to decide whether to add new records into the buffer.
+   */
+  private transient Condition addToBufferCondition;
+
+  /**
+   * Flag saying whether there is an on-going checkpoint.
+   */
+  private volatile boolean onCheckpointing = false;
+
+  /**
+   * Config options.
+   */
+  private final Configuration config;
+
+  /**
+   * Id of current subtask.
+   */
+  private int taskID;
+
+  /**
+   * Write Client.
+   */
+  private transient HoodieFlinkWriteClient writeClient;
+
+  private transient BiFunction<List<HoodieRecord>, String, List<WriteStatus>> 
writeFunction;
+
+  /**
+   * HoodieKey generator.
+   */
+  private transient KeyGenerator keyGenerator;
+
+  /**
+   * Row type of the input.
+   */
+  private final RowType rowType;
+
+  /**
+   * Avro schema of the input.
+   */
+  private final Schema avroSchema;
+
+  private transient RowDataToAvroConverters.RowDataToAvroConverter converter;
+
+  /**
+   * The REQUESTED instant we write the data.
+   */
+  private volatile String currentInstant;
+
+  /**
+   * Gateway to send operator events to the operator coordinator.
+   */
+  private transient OperatorEventGateway eventGateway;
+
+  /**
+   * Constructs a StreamingSinkFunction.
+   *
+   * @param rowType The input row type
+   * @param config  The config options
+   */
+  public StreamWriteFunction(RowType rowType, Configuration config) {

Review comment:
       > This brings an additional burden for sharing the same abstraction
   The main abstraction that we can share is the partitioning and 
micro-batching, not the source record parsing, so this should not be a concern. 
On the contrary, the `RowData` is very efficient in processing and it supports 
many data formats, e.g. `JSON`/`CSV`/`Avro` which are all complex and have many 
config options.
   
   > RowType makes the number of compatible Flink versions less
   The old pipeline does not use the `RowType`, so it should not have 
compatibility problem.
   
   BTW, the Hoodie `Spark` data source also use `Row` as the basic data 
structure, which is easy for `JSON`/`CSV`/`Avro` source data conversion and 
effective data handling in the pipeline.
   

##########
File path: 
hudi-flink/src/main/java/org/apache/hudi/operator/StreamWriteFunction.java
##########
@@ -0,0 +1,344 @@
+/*
+ * 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.hudi.avro.HoodieAvroUtils;
+import org.apache.hudi.client.FlinkTaskContextSupplier;
+import org.apache.hudi.client.HoodieFlinkWriteClient;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.client.common.HoodieFlinkEngineContext;
+import org.apache.hudi.common.config.SerializableConfiguration;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.common.model.WriteOperationType;
+import org.apache.hudi.common.util.ObjectSizeCalculator;
+import org.apache.hudi.keygen.KeyGenerator;
+import org.apache.hudi.operator.event.BatchWriteSuccessEvent;
+import org.apache.hudi.util.StreamerUtil;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.formats.avro.RowDataToAvroConverters;
+import org.apache.flink.runtime.operators.coordination.OperatorEventGateway;
+import org.apache.flink.runtime.state.FunctionInitializationContext;
+import org.apache.flink.runtime.state.FunctionSnapshotContext;
+import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
+import org.apache.flink.streaming.api.functions.KeyedProcessFunction;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.util.Collector;
+import org.apache.flink.util.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Random;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.function.BiFunction;
+
+/**
+ * Sink function to write the data to the underneath filesystem.
+ *
+ * <p><h2>Work Flow</h2>
+ *
+ * <p>The function firstly buffers the data as a batch of {@link 
HoodieRecord}s,
+ * It flushes(write) the records batch when a Flink checkpoint starts. After a 
batch has been written successfully,
+ * the function notifies its operator coordinator {@link 
StreamWriteOperatorCoordinator} to mark a successful write.
+ *
+ * <p><h2>Exactly-once Semantics</h2>
+ *
+ * <p>The task implements exactly-once semantics by buffering the data between 
checkpoints. The operator coordinator
+ * starts a new instant on the time line when a checkpoint triggers, the 
coordinator checkpoints always
+ * start before its operator, so when this function starts a checkpoint, a 
REQUESTED instant already exists.
+ * The function process thread then block data buffering and the checkpoint 
thread starts flushing the existing data buffer.
+ * When the existing data buffer write successfully, the process thread 
unblock and start buffering again for the next round checkpoint.
+ * Because any checkpoint failures would trigger the write rollback, it 
implements the exactly-once semantics.
+ *
+ * <p><h2>Fault Tolerance</h2>
+ *
+ * <p>The operator coordinator checks the validity for the last instant when 
it starts a new one. The operator rolls back
+ * the written data and throws when any error occurs. This means any 
checkpoint or task failure would trigger a failover.
+ * The operator coordinator would try several times when committing the 
writestatus.
+ *
+ * <p>Note: The function task requires the input stream be partitioned by the 
partition fields to avoid different write tasks
+ * write to the same file group that conflict. The general case for partition 
path is a datetime field,
+ * so the sink task is very possible to have IO bottleneck, the more flexible 
solution is to shuffle the
+ * data by the file group IDs.
+ *
+ * @param <I> Type of the input record
+ * @see StreamWriteOperatorCoordinator
+ */
+public class StreamWriteFunction<K, I, O> extends KeyedProcessFunction<K, I, 
O> implements CheckpointedFunction {
+
+  private static final long serialVersionUID = 1L;
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(StreamWriteFunction.class);
+
+  /**
+   * Write buffer for a checkpoint.
+   */
+  private transient List<HoodieRecord> buffer;
+
+  /**
+   * The buffer lock to control data buffering/flushing.
+   */
+  private transient ReentrantLock bufferLock;
+
+  /**
+   * The condition to decide whether to add new records into the buffer.
+   */
+  private transient Condition addToBufferCondition;
+
+  /**
+   * Flag saying whether there is an on-going checkpoint.
+   */
+  private volatile boolean onCheckpointing = false;
+
+  /**
+   * Config options.
+   */
+  private final Configuration config;
+
+  /**
+   * Id of current subtask.
+   */
+  private int taskID;
+
+  /**
+   * Write Client.
+   */
+  private transient HoodieFlinkWriteClient writeClient;
+
+  private transient BiFunction<List<HoodieRecord>, String, List<WriteStatus>> 
writeFunction;
+
+  /**
+   * HoodieKey generator.
+   */
+  private transient KeyGenerator keyGenerator;
+
+  /**
+   * Row type of the input.
+   */
+  private final RowType rowType;
+
+  /**
+   * Avro schema of the input.
+   */
+  private final Schema avroSchema;
+
+  private transient RowDataToAvroConverters.RowDataToAvroConverter converter;
+
+  /**
+   * The REQUESTED instant we write the data.
+   */
+  private volatile String currentInstant;
+
+  /**
+   * Gateway to send operator events to the operator coordinator.
+   */
+  private transient OperatorEventGateway eventGateway;
+
+  /**
+   * Constructs a StreamingSinkFunction.
+   *
+   * @param rowType The input row type
+   * @param config  The config options
+   */
+  public StreamWriteFunction(RowType rowType, Configuration config) {

Review comment:
       > This brings an additional burden for sharing the same abstraction
   
   The main abstraction that we can share is the partitioning and 
micro-batching, not the source record parsing, so this should not be a concern. 
On the contrary, the `RowData` is very efficient in processing and it supports 
many data formats, e.g. `JSON`/`CSV`/`Avro` which are all complex and have many 
config options.
   
   > RowType makes the number of compatible Flink versions less
   
   The old pipeline does not use the `RowType`, so it should not have 
compatibility problem.
   
   BTW, the Hoodie `Spark` data source also use `Row` as the basic data 
structure, which is easy for `JSON`/`CSV`/`Avro` source data conversion and 
effective data handling in the pipeline.
   

##########
File path: 
hudi-flink/src/main/java/org/apache/hudi/operator/event/BatchWriteSuccessEvent.java
##########
@@ -0,0 +1,64 @@
+/*
+ * 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.event;
+
+import org.apache.flink.runtime.operators.coordination.OperatorEvent;
+
+import org.apache.hudi.client.WriteStatus;
+
+import java.util.List;
+
+/**
+ * An operator even to mark successful checkpoint batch write.
+ */
+public class BatchWriteSuccessEvent implements OperatorEvent {
+  private static final long serialVersionUID = 1L;
+
+  private final List<WriteStatus> writeStatuses;
+  private final int taskID;
+  private final String instantTime;
+
+  /**
+   * Creates an event.
+   *
+   * @param taskID        The task ID
+   * @param instantTime   The instant time under which to write the data
+   * @param writeStatuses The write statues list
+   */
+  public BatchWriteSuccessEvent(

Review comment:
       Removed.

##########
File path: 
hudi-flink/src/main/java/org/apache/hudi/streamer/FlinkStreamerConfig.java
##########
@@ -0,0 +1,124 @@
+/*
+ * 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.streamer;
+
+import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload;
+import org.apache.hudi.common.model.WriteOperationType;
+import org.apache.hudi.keygen.SimpleAvroKeyGenerator;
+
+import com.beust.jcommander.Parameter;
+import org.apache.flink.configuration.Configuration;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Configurations for Hoodie Flink streamer.
+ */
+public class FlinkStreamerConfig extends Configuration {
+  @Parameter(names = {"--kafka-topic"}, description = "Kafka topic name.", 
required = true)
+  public String kafkaTopic;
+
+  @Parameter(names = {"--kafka-group-id"}, description = "Kafka consumer group 
id.", required = true)
+  public String kafkaGroupId;
+
+  @Parameter(names = {"--kafka-bootstrap-servers"}, description = "Kafka 
bootstrap.servers.", required = true)
+  public String kafkaBootstrapServers;
+
+  @Parameter(names = {"--flink-checkpoint-path"}, description = "Flink 
checkpoint path.")
+  public String flinkCheckPointPath;
+
+  @Parameter(names = {"--flink-block-retry-times"}, description = "Times to 
retry when latest instant has not completed.")

Review comment:
       Copied from the old code, how about `instant-retry-times` ?

##########
File path: 
hudi-flink/src/main/java/org/apache/hudi/streamer/HoodieFlinkStreamerV2.java
##########
@@ -0,0 +1,103 @@
+/*
+ * 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.streamer;
+
+import org.apache.hudi.operator.FlinkOptions;
+import org.apache.hudi.operator.StreamWriteOperatorFactory;
+import org.apache.hudi.util.StreamerUtil;
+
+import com.beust.jcommander.JCommander;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.formats.avro.typeutils.AvroSchemaConverter;
+import org.apache.flink.formats.json.JsonRowDataDeserializationSchema;
+import org.apache.flink.formats.json.TimestampFormat;
+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.flink.table.data.RowData;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.RowType;
+
+import java.util.Properties;
+
+/**
+ * An Utility which can incrementally consume data from Kafka and apply it to 
the target table.
+ * currently, it only support COW table and insert, upsert operation.
+ */
+public class HoodieFlinkStreamerV2 {
+  public static void main(String[] args) throws Exception {
+    StreamExecutionEnvironment env = 
StreamExecutionEnvironment.getExecutionEnvironment();
+
+    final FlinkStreamerConfig cfg = new FlinkStreamerConfig();
+    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);
+    // We use checkpoint to trigger write operation, including instant 
generating and committing,
+    // There can only be one checkpoint at one time.
+    env.getCheckpointConfig().setMaxConcurrentCheckpoints(1);
+    env.disableOperatorChaining();

Review comment:
       Not necessary, removed.

##########
File path: hudi-flink/src/main/java/org/apache/hudi/operator/FlinkOptions.java
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.streamer.FlinkStreamerConfig;
+import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload;
+import org.apache.hudi.keygen.SimpleAvroKeyGenerator;
+import org.apache.hudi.keygen.constant.KeyGeneratorOptions;
+import org.apache.hudi.util.StreamerUtil;
+
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ConfigOptions;
+import org.apache.flink.configuration.Configuration;
+
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Hoodie Flink config options.
+ *
+ * <p>It has the options for Hoodie table read and write. It also defines some 
utilities.
+ */
+public class FlinkOptions {
+  private FlinkOptions() {
+  }
+
+  // ------------------------------------------------------------------------
+  //  Base Options
+  // ------------------------------------------------------------------------
+  public static final ConfigOption<String> PATH = ConfigOptions
+      .key("path")
+      .stringType()
+      .noDefaultValue()
+      .withDescription("Base path for the target hoodie table."
+          + "\nThe path would be created if it does not exist,\n"
+          + "otherwise a Hoodie table expects to be initialized successfully");
+
+  public static final ConfigOption<String> PROPS_FILE_PATH = ConfigOptions
+      .key("properties-file.path")
+      .stringType()
+      .noDefaultValue()
+      .withDescription("Path to properties file on local-fs or dfs, with 
configurations for \n"
+          + "hoodie client, schema provider, key generator and data source. 
For hoodie client props, sane defaults are\n"
+          + "used, but recommend use to provide basic things like metrics 
endpoints, hive configs etc. For sources, refer\n"
+          + "to individual classes, for supported properties");
+
+  // ------------------------------------------------------------------------
+  //  Read Options
+  // ------------------------------------------------------------------------
+  public static final ConfigOption<String> READ_SCHEMA_FILE_PATH = 
ConfigOptions
+      .key("read.schema.file.path")
+      .stringType()
+      .noDefaultValue()
+      .withDescription("Avro schema file path, the parsed schema is used for 
deserializing");
+
+  // ------------------------------------------------------------------------
+  //  Write Options
+  // ------------------------------------------------------------------------
+  public static final ConfigOption<String> TABLE_NAME = ConfigOptions
+      .key(HoodieWriteConfig.TABLE_NAME)
+      .stringType()
+      .noDefaultValue()
+      .withDescription("Table name to register to Hive metastore");
+
+  public static final ConfigOption<String> TABLE_TYPE = ConfigOptions
+      .key("write.table.type")
+      .stringType()
+      .defaultValue("COPY_ON_WRITE")
+      .withDescription("Type of table to write. COPY_ON_WRITE (or) 
MERGE_ON_READ");
+
+  public static final ConfigOption<String> OPERATION = ConfigOptions
+      .key("write.operation")
+      .stringType()
+      .defaultValue("upsert")
+      .withDescription("The write operation, that this write should do");
+
+  public static final ConfigOption<String> PRECOMBINE_FIELD = ConfigOptions
+      .key("write.precombine.field")

Review comment:
       We can, but i would suggest to do this is a separate issue, we need to 
define some rules for config option name.

##########
File path: 
hudi-flink/src/main/java/org/apache/hudi/operator/StreamWriteFunction.java
##########
@@ -0,0 +1,342 @@
+/*
+ * 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.hudi.avro.HoodieAvroUtils;
+import org.apache.hudi.client.FlinkTaskContextSupplier;
+import org.apache.hudi.client.HoodieFlinkWriteClient;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.client.common.HoodieFlinkEngineContext;
+import org.apache.hudi.common.config.SerializableConfiguration;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.common.model.WriteOperationType;
+import org.apache.hudi.common.util.ObjectSizeCalculator;
+import org.apache.hudi.keygen.KeyGenerator;
+import org.apache.hudi.operator.event.BatchWriteSuccessEvent;
+import org.apache.hudi.util.StreamerUtil;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.formats.avro.RowDataToAvroConverters;
+import org.apache.flink.runtime.operators.coordination.OperatorEventGateway;
+import org.apache.flink.runtime.state.FunctionInitializationContext;
+import org.apache.flink.runtime.state.FunctionSnapshotContext;
+import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
+import org.apache.flink.streaming.api.functions.KeyedProcessFunction;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.util.Collector;
+import org.apache.flink.util.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Random;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.function.BiFunction;
+
+/**
+ * Sink function to write the data to the underneath filesystem.
+ *
+ * <p><h2>Work Flow</h2>
+ *
+ * <p>The function firstly buffers the data as a batch of {@link 
HoodieRecord}s,
+ * It flushes(write) the records batch when a Flink checkpoint starts. After a 
batch has been written successfully,
+ * the function notifies its operator coordinator {@link 
StreamWriteOperatorCoordinator} to mark a successful write.
+ *
+ * <p><h2>Exactly-once Semantics</h2>
+ *
+ * <p>The task implements exactly-once semantics by buffering the data between 
checkpoints. The operator coordinator
+ * starts a new instant on the time line when a checkpoint triggers, the 
coordinator checkpoints always
+ * start before its operator, so when this function starts a checkpoint, a 
REQUESTED instant already exists.
+ * The function process thread then block data buffering and the checkpoint 
thread starts flushing the existing data buffer.
+ * When the existing data buffer write successfully, the process thread 
unblock and start buffering again for the next round checkpoint.
+ * Because any checkpoint failures would trigger the write rollback, it 
implements the exactly-once semantics.
+ *
+ * <p><h2>Fault Tolerance</h2>
+ *
+ * <p>The operator coordinator checks the validity for the last instant when 
it starts a new one. The operator rolls back
+ * the written data and throws when any error occurs. This means any 
checkpoint or task failure would trigger a failover.
+ * The operator coordinator would try several times when committing the 
writestatus.
+ *
+ * <p>Note: The function task requires the input stream be partitioned by the 
partition fields to avoid different write tasks
+ * write to the same file group that conflict. The general case for partition 
path is a datetime field,
+ * so the sink task is very possible to have IO bottleneck, the more flexible 
solution is to shuffle the
+ * data by the file group IDs.
+ *
+ * @param <I> Type of the input record
+ * @see StreamWriteOperatorCoordinator
+ */
+public class StreamWriteFunction<K, I, O>
+    extends KeyedProcessFunction<K, I, O>
+    implements CheckpointedFunction {
+
+  private static final long serialVersionUID = 1L;
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(StreamWriteFunction.class);
+
+  /**
+   * Write buffer for a checkpoint.
+   */
+  private transient List<HoodieRecord> buffer;
+
+  /**
+   * The buffer lock to control data buffering/flushing.
+   */
+  private transient ReentrantLock bufferLock;
+
+  /**
+   * The condition to decide whether to add new records into the buffer.
+   */
+  private transient Condition addToBufferCondition;
+
+  /**
+   * Flag saying whether there is an on-going checkpoint.
+   */
+  private volatile boolean onCheckpointing = false;
+
+  /**
+   * Config options.
+   */
+  private final Configuration config;
+
+  /**
+   * Id of current subtask.
+   */
+  private int taskID;
+
+  /**
+   * Write Client.
+   */
+  private transient HoodieFlinkWriteClient writeClient;
+
+  private transient BiFunction<List<HoodieRecord>, String, List<WriteStatus>> 
writeFunction;
+
+  /**
+   * HoodieKey generator.
+   */
+  private transient KeyGenerator keyGenerator;
+
+  /**
+   * Row type of the input.
+   */
+  private final RowType rowType;
+
+  /**
+   * Avro schema of the input.
+   */
+  private final Schema avroSchema;
+
+  private transient RowDataToAvroConverters.RowDataToAvroConverter converter;
+
+  /**
+   * The REQUESTED instant we write the data.
+   */
+  private volatile String currentInstant;
+
+  /**
+   * Gateway to send operator events to the operator coordinator.
+   */
+  private transient OperatorEventGateway eventGateway;
+
+  /**
+   * Constructs a StreamingSinkFunction.
+   *
+   * @param rowType The input row type
+   * @param config  The config options
+   */
+  public StreamWriteFunction(RowType rowType, Configuration config) {
+    this.rowType = rowType;
+    this.avroSchema = org.apache.flink.formats.avro.typeutils
+        .AvroSchemaConverter.convertToSchema(rowType);
+    this.config = config;
+  }
+
+  @Override
+  public void open(Configuration parameters) throws IOException {
+    this.taskID = getRuntimeContext().getIndexOfThisSubtask();
+    this.keyGenerator = 
StreamerUtil.createKeyGenerator(HoodieOptions.flatOptions(this.config));
+    this.converter = RowDataToAvroConverters.createConverter(this.rowType);
+    initBuffer();
+    initWriteClient();
+    initWriteFunction();
+  }
+
+  @Override
+  public void initializeState(FunctionInitializationContext context) {
+    // no operation
+  }
+
+  @Override
+  public void snapshotState(FunctionSnapshotContext functionSnapshotContext) {
+    bufferLock.lock();
+    try {
+      // Based on the fact that the coordinator starts the checkpoint first,
+      // it would check the validity.
+      this.onCheckpointing = true;
+      this.currentInstant = 
this.writeClient.getInflightAndRequestedInstant(this.config.get(HoodieOptions.TABLE_TYPE));
+      Preconditions.checkNotNull(this.currentInstant,
+          "No inflight instant when flushing data");
+      // wait for the buffer data flush out and request a new instant
+      flushBuffer();
+      // signal the task thread to start buffering
+      addToBufferCondition.signal();
+    } finally {
+      this.onCheckpointing = false;
+      bufferLock.unlock();
+    }
+  }
+
+  @Override
+  public void processElement(I value, KeyedProcessFunction<K, I, O>.Context 
ctx, Collector<O> out) throws Exception {
+    bufferLock.lock();
+    try {
+      if (onCheckpointing) {
+        addToBufferCondition.await();
+      }
+      this.buffer.add(toHoodie(value));
+    } finally {
+      bufferLock.unlock();
+    }
+  }
+
+  @Override
+  public void close() {
+    if (this.writeClient != null) {
+      this.writeClient.close();
+    }
+  }
+
+  // -------------------------------------------------------------------------
+  //  Getter/Setter
+  // -------------------------------------------------------------------------
+
+  @VisibleForTesting
+  @SuppressWarnings("rawtypes")
+  public List<HoodieRecord> getBuffer() {
+    return buffer;
+  }
+
+  @VisibleForTesting
+  @SuppressWarnings("rawtypes")
+  public HoodieFlinkWriteClient getWriteClient() {
+    return writeClient;
+  }
+
+  public void setOperatorEventGateway(OperatorEventGateway 
operatorEventGateway) {
+    this.eventGateway = operatorEventGateway;
+  }
+
+  // -------------------------------------------------------------------------
+  //  Utilities
+  // -------------------------------------------------------------------------
+
+  private void initBuffer() {
+    this.buffer = new ArrayList<>();
+    this.bufferLock = new ReentrantLock();
+    this.addToBufferCondition = this.bufferLock.newCondition();
+  }
+
+  private void initWriteClient() {
+    HoodieFlinkEngineContext context =
+        new HoodieFlinkEngineContext(
+            new SerializableConfiguration(StreamerUtil.getHadoopConf()),
+            new FlinkTaskContextSupplier(getRuntimeContext()));
+
+    writeClient = new HoodieFlinkWriteClient<>(context, 
StreamerUtil.getHoodieClientConfig(this.config));
+  }
+
+  private void initWriteFunction() {
+    final String writeOperation = this.config.get(HoodieOptions.OPERATION);
+    switch (WriteOperationType.fromValue(writeOperation)) {
+      case INSERT:
+        this.writeFunction = (records, instantTime) -> 
this.writeClient.insert(records, instantTime);
+        break;
+      case UPSERT:
+        this.writeFunction = (records, instantTime) -> 
this.writeClient.upsert(records, instantTime);
+        break;
+      default:
+        throw new RuntimeException("Unsupported write operation : " + 
writeOperation);
+    }
+  }
+
+  // Keep for mini-batch write.
+  private static class BufferSizeEstimator {

Review comment:
       Removed.

##########
File path: hudi-flink/src/test/resources/test_source.data
##########
@@ -0,0 +1,8 @@
+{"uuid": "id1", "name": "Danny", "age": 23, "ts": "1970-01-01T00:00:01", 
"partition": "par1"}

Review comment:
       No, i'm old




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