You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hudi.apache.org by GitBox <gi...@apache.org> on 2021/01/16 16:27:57 UTC

[GitHub] [hudi] garyli1019 commented on a change in pull request #2430: [HUDI-1522] Add a new pipeline for Flink writer

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



##########
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:
       This data also available in the `WriteStatus`. We have `TotalWriteBytes` and `TotalRecords` e.t.c. How are we gonna use this later on?

##########
File path: hudi-flink/src/main/java/org/apache/hudi/operator/HoodieOptions.java
##########
@@ -0,0 +1,248 @@
+/*
+ * 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.streamer.Config;
+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 HoodieOptions {
+  private HoodieOptions() {
+  }
+
+  // ------------------------------------------------------------------------
+  //  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("write.table.name")

Review comment:
       Can we reuse the config under `HoodieWriteConfiguration`? That would be great if we can use the same set of config for both Spark and Flink.

##########
File path: hudi-flink/src/main/java/org/apache/hudi/operator/HoodieOptions.java
##########
@@ -0,0 +1,248 @@
+/*
+ * 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.streamer.Config;
+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 HoodieOptions {

Review comment:
       +1 for adding `Flink` into the class name. We might add more engine support in the future. Use `HoodieOptions` make more sense to me if we add this connector under Flink's codebase.

##########
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 {
+    private final Random random = new Random(47);
+
+    private long lastRecordSize = -1L;
+    private long totalSize = 0L;
+
+    long estimate(Object record) {
+      if (lastRecordSize == -1 || sampling()) {
+        lastRecordSize = ObjectSizeCalculator.getObjectSize(record);
+      }
+      totalSize += lastRecordSize;
+      return totalSize;
+    }
+
+    boolean sampling() {
+      // 0.01 sampling percentage
+      return random.nextInt(100) == 1;
+    }
+
+    void reset() {
+      this.lastRecordSize = -1L;
+      this.totalSize = 0L;
+    }
+  }
+
+  /**
+   * Converts the give record to a {@link HoodieRecord}.
+   *
+   * @param record The input record
+   * @return HoodieRecord based on the configuration
+   * @throws IOException if error occurs
+   */
+  @SuppressWarnings("rawtypes")
+  private HoodieRecord toHoodie(I record) throws IOException {

Review comment:
       nit: `toHoodieRecord`?

##########
File path: hudi-flink/src/test/java/org/apache/hudi/operator/utils/StreamWriteFunctionWrapper.java
##########
@@ -0,0 +1,122 @@
+/*
+ * 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.utils;
+
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.io.disk.iomanager.IOManager;
+import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
+import org.apache.flink.runtime.memory.MemoryManager;
+import org.apache.flink.runtime.operators.coordination.OperatorEvent;
+import org.apache.flink.runtime.operators.testutils.MockEnvironment;
+import org.apache.flink.runtime.operators.testutils.MockEnvironmentBuilder;
+import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
+import org.apache.flink.streaming.api.operators.collect.utils.MockFunctionSnapshotContext;
+import org.apache.flink.streaming.api.operators.collect.utils.MockOperatorEventGateway;
+
+import org.apache.hudi.client.HoodieFlinkWriteClient;
+import org.apache.hudi.operator.StreamWriteFunction;
+import org.apache.hudi.operator.StreamWriteOperatorCoordinator;
+import org.apache.hudi.operator.event.BatchWriteSuccessEvent;
+
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * A wrapper class to manipulate the {@link StreamWriteFunction} instance for testing.
+ *
+ * @param <I> Input type
+ */
+public class StreamWriteFunctionWrapper<I> {
+  private final TypeSerializer<I> serializer;
+  private final Configuration conf;
+
+  private final IOManager ioManager;
+  private final StreamingRuntimeContext runtimeContext;
+  private final MockOperatorEventGateway gateway;
+  private final StreamWriteOperatorCoordinator coordinator;
+  private final MockFunctionInitializationContext functionInitializationContext;
+
+  private StreamWriteFunction<Object, I, Object> function;
+
+  public StreamWriteFunctionWrapper(String tablePath, TypeSerializer<I> serializer) throws Exception {
+    this.serializer = serializer;
+    this.ioManager = new IOManagerAsync();
+    MockEnvironment environment = new MockEnvironmentBuilder()
+        .setTaskName("mockTask")
+        .setManagedMemorySize(4 * MemoryManager.DEFAULT_PAGE_SIZE)
+        .setIOManager(ioManager)
+        .build();
+    this.runtimeContext = new MockStreamingRuntimeContext(false, 1, 0, environment);
+    this.gateway = new MockOperatorEventGateway();
+    this.conf = TestConfigurations.getDefaultConf(tablePath);
+    // one function
+    this.coordinator = new StreamWriteOperatorCoordinator(conf, 1);
+    this.coordinator.start();
+    this.functionInitializationContext = new MockFunctionInitializationContext();
+  }
+
+  public void openFunction() throws Exception {
+    function = new StreamWriteFunction<>(TestConfigurations.ROW_TYPE, this.conf);
+    function.setRuntimeContext(runtimeContext);
+    function.setOperatorEventGateway(gateway);
+    function.open(this.conf);
+  }
+
+  public void invoke(I record) throws Exception {
+    function.processElement(record, null, null);
+  }
+
+  public BatchWriteSuccessEvent[] getEventBuffer() {
+    return this.coordinator.getEventBuffer();
+  }
+
+  public OperatorEvent getNextEvent() {
+    return this.gateway.getNextEvent();
+  }
+
+  @SuppressWarnings("rawtypes")
+  public HoodieFlinkWriteClient getWriteClient() {
+    return this.function.getWriteClient();
+  }
+
+  public void checkpointFunction(long checkpointId) throws Exception {

Review comment:
       Where is this method being called? How do we generate the checkpointId?

##########
File path: hudi-flink/src/main/java/org/apache/hudi/util/StreamerUtil.java
##########
@@ -109,6 +167,21 @@ public static KeyGenerator createKeyGenerator(TypedProperties props) throws IOEx
     }
   }
 
+  /**
+   * Create a key generator class via reflection, passing in any configs needed.
+   * <p>
+   * If the class name of key generator is configured through the properties file, i.e., {@code props}, use the corresponding key generator class; otherwise, use the default key generator class
+   * specified in {@code DataSourceWriteOptions}.

Review comment:
       nit: this linked to Spark code

##########
File path: hudi-flink/src/main/java/org/apache/hudi/operator/StreamWriteOperatorFactory.java
##########
@@ -0,0 +1,79 @@
+/*
+ * 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.configuration.Configuration;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.runtime.operators.coordination.OperatorCoordinator;
+import org.apache.flink.runtime.operators.coordination.OperatorEventDispatcher;
+import org.apache.flink.streaming.api.operators.CoordinatedOperatorFactory;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory;
+import org.apache.flink.streaming.api.operators.SimpleUdfStreamOperatorFactory;
+import org.apache.flink.streaming.api.operators.StreamOperator;
+import org.apache.flink.streaming.api.operators.StreamOperatorParameters;
+import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService;
+import org.apache.flink.table.types.logical.RowType;
+
+/**
+ * Factory class for {@link StreamWriteOperator}.
+ */
+public class StreamWriteOperatorFactory<I>
+    extends SimpleUdfStreamOperatorFactory<Object>
+    implements CoordinatedOperatorFactory<Object>, OneInputStreamOperatorFactory<I, Object> {
+  private static final long serialVersionUID = 1L;
+
+  private static final long DEFAULT_MAX_BYTES_PER_BATCH = (1 << 21) * 128; // 256MB

Review comment:
       For the later use? 

##########
File path: hudi-flink/src/main/java/org/apache/hudi/streamer/OperationConverter.java
##########
@@ -0,0 +1,34 @@
+/*
+ * 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.WriteOperationType;
+
+import com.beust.jcommander.IStringConverter;
+import com.beust.jcommander.ParameterException;
+
+/**
+ * Converter that converts a string into enum WriteOperationType.
+ */
+public class OperationConverter implements IStringConverter<WriteOperationType> {
+  @Override
+  public WriteOperationType convert(String value) throws ParameterException {

Review comment:
       This is available in the `WriteOperationType`. Can we reuse from there?

##########
File path: hudi-flink/src/main/java/org/apache/hudi/operator/StreamWriteOperatorCoordinator.java
##########
@@ -0,0 +1,419 @@
+/*
+ * 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.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.fs.FSUtils;
+import org.apache.hudi.common.model.HoodieTableType;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.operator.event.BatchWriteSuccessEvent;
+import org.apache.hudi.util.StreamerUtil;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.core.memory.DataInputViewStreamWrapper;
+import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.runtime.operators.coordination.OperatorCoordinator;
+import org.apache.flink.runtime.operators.coordination.OperatorEvent;
+import org.apache.flink.util.Preconditions;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.jetbrains.annotations.Nullable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Objects;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+/**
+ * {@link OperatorCoordinator} for {@link StreamWriteFunction}.
+ *
+ * <p>This coordinator starts a new instant when a new checkpoint starts. It commits the instant when all the
+ * operator tasks write the buffer successfully for a round of checkpoint.
+ *
+ * <p>If there is no data for a round of checkpointing, it rolls back the metadata.
+ *
+ * @see StreamWriteFunction for the work flow and semantics
+ */
+public class StreamWriteOperatorCoordinator
+    implements OperatorCoordinator {
+  private static final Logger LOG = LoggerFactory.getLogger(StreamWriteOperatorCoordinator.class);
+
+  /**
+   * Config options.
+   */
+  private final Configuration conf;
+
+  /**
+   * Write client.
+   */
+  private transient HoodieFlinkWriteClient writeClient;
+
+  private long inFlightCheckpoint = -1;
+
+  /**
+   * Current REQUESTED instant, for validation.
+   */
+  private String inFlightInstant = "";
+
+  /**
+   * Event buffer for one round of checkpointing. When all the elements are non-null and have the same
+   * write instant, then the instant succeed and we can commit it.
+   */
+  private transient BatchWriteSuccessEvent[] eventBuffer;
+
+  /**
+   * Task number of the operator.
+   */
+  private final int parallelism;
+
+  /**
+   * Constructs a StreamingSinkOperatorCoordinator.
+   *
+   * @param conf        The config options
+   * @param parallelism The operator task number
+   */
+  public StreamWriteOperatorCoordinator(
+      Configuration conf,
+      int parallelism) {
+    this.conf = conf;
+    this.parallelism = parallelism;
+  }
+
+  @Override
+  public void start() throws Exception {
+    // initialize event buffer
+    reset();
+    // writeClient
+    initWriteClient();
+    // init table, create it if not exists.
+    initTable();
+  }
+
+  @Override
+  public void close() {
+    if (writeClient != null) {
+      writeClient.close();
+    }
+    this.eventBuffer = null;
+  }
+
+  @Override
+  public void checkpointCoordinator(long checkpointId, CompletableFuture<byte[]> result) {
+    try {
+      final String errMsg = "A new checkpoint starts while the last checkpoint buffer"
+          + " data has not finish writing, roll back the last write and throw";
+      checkAndForceCommit(errMsg);
+      this.inFlightInstant = this.writeClient.startCommit();
+      this.inFlightCheckpoint = checkpointId;
+      LOG.info("Create instant [{}], at checkpoint [{}]", this.inFlightInstant, checkpointId);
+      result.complete(writeCheckpointBytes());
+    } catch (Throwable throwable) {
+      // when a checkpoint fails, throws directly.
+      result.completeExceptionally(
+          new CompletionException(

Review comment:
       How about using a hoodie internal exception here?




----------------------------------------------------------------
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:
users@infra.apache.org