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/11 07:32:32 UTC

[GitHub] [hudi] danny0405 opened a new pull request #2430: [HUDI-1522] Remove the single parallelism operator from the Flink writer

danny0405 opened a new pull request #2430:
URL: https://github.com/apache/hudi/pull/2430


   pipeline
   
   This is the #step 1 of RFC-24:
   https://cwiki.apache.org/confluence/display/HUDI/RFC+-+24%3A+Hoodie+Flink+Writer+Proposal
   
   * Remove the InstantGenerateOperator and CommitSink, add
     StreamWriteOperatorCoordinator to achieve the same function
   * Add HoodieOptions for configuration of all the Flink operators, this
     is also useful for the SQL connectors which would be introduced in
     following contributions
   * Add UT and IT cases for the new operators
   
   ## *Tips*
   - *Thank you very much for contributing to Apache Hudi.*
   - *Please review https://hudi.apache.org/contributing.html before opening a pull request.*
   
   ## What is the purpose of the pull request
   
   *(For example: This pull request adds quick-start document.)*
   
   ## Brief change log
   
   *(for example:)*
     - *Modify AnnotationLocation checkstyle rule in checkstyle.xml*
   
   ## Verify this pull request
   
   *(Please pick either of the following options)*
   
   This pull request is a trivial rework / code cleanup without any test coverage.
   
   *(or)*
   
   This pull request is already covered by existing tests, such as *(please describe tests)*.
   
   (or)
   
   This change added tests and can be verified as follows:
   
   *(example:)*
   
     - *Added integration tests for end-to-end.*
     - *Added HoodieClientWriteTest to verify the change.*
     - *Manually verified the change by running a job locally.*
   
   ## Committer checklist
   
    - [ ] Has a corresponding JIRA in PR title & commit
    
    - [ ] Commit message is descriptive of the change
    
    - [ ] CI is green
   
    - [ ] Necessary doc changes done or have another open PR
          
    - [ ] For large changes, please consider breaking it into sub-tasks under an umbrella JIRA.


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



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

Posted by GitBox <gi...@apache.org>.
danny0405 commented on a change in pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#discussion_r560730466



##########
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")
+      .stringType()
+      .defaultValue("ts")
+      .withDescription("Field used in preCombining before actual write. When two records have the same\n"
+          + "key value, we will pick the one with the largest value for the precombine field,\n"
+          + "determined by Object.compareTo(..)");
+
+  public static final ConfigOption<String> PAYLOAD_CLASS = ConfigOptions
+      .key("write.payload.class")
+      .stringType()
+      .defaultValue(OverwriteWithLatestAvroPayload.class.getName())
+      .withDescription("Payload class used. Override this, if you like to roll your own merge logic, when upserting/inserting.\n"
+          + "This will render any value set for the option in-effective");
+
+  /**
+   * Flag to indicate whether to drop duplicates upon insert.
+   * By default insert will accept duplicates, to gain extra performance.
+   */
+  public static final ConfigOption<Boolean> INSERT_DROP_DUPS = ConfigOptions
+      .key("write.insert.drop.duplicates")
+      .booleanType()
+      .defaultValue(false)
+      .withDescription("Flag to indicate whether to drop duplicates upon insert.\n"
+          + "By default insert will accept duplicates, to gain extra performance");
+
+  public static final ConfigOption<Integer> RETRY_TIMES = ConfigOptions
+      .key("write.retry.times")
+      .intType()
+      .defaultValue(3)
+      .withDescription("Flag to indicate how many times streaming job should retry for a failed checkpoint batch.\n"
+          + "By default 3");
+
+  public static final ConfigOption<Long> RETRY_INTERVAL_MS = ConfigOptions
+      .key("write.retry.interval.ms")
+      .longType()
+      .defaultValue(2000L)
+      .withDescription("Flag to indicate how long (by millisecond) before a retry should issued for failed checkpoint batch.\n"
+          + "By default 2000 and it will be doubled by every retry");
+
+  public static final ConfigOption<Boolean> IGNORE_FAILED_BATCH = ConfigOptions
+      .key("write.ignore.failed.batch")
+      .booleanType()
+      .defaultValue(true)
+      .withDescription("Flag to indicate whether to ignore any non exception error (e.g. writestatus error). within a checkpoint batch.\n"
+          + "By default true (in favor of streaming progressing over data integrity)");
+
+  public static final ConfigOption<String> RECORD_KEY_FIELD = ConfigOptions
+      .key(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY)
+      .stringType()
+      .defaultValue("uuid")
+      .withDescription("Record key field. Value to be used as the `recordKey` component of `HoodieKey`.\n"
+          + "Actual value will be obtained by invoking .toString() on the field value. Nested fields can be specified using "
+          + "the dot notation eg: `a.b.c`");
+
+  public static final ConfigOption<String> PARTITION_PATH_FIELD = ConfigOptions
+      .key(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY)
+      .stringType()
+      .defaultValue("partition-path")
+      .withDescription("Partition path field. Value to be used at the `partitionPath` component of `HoodieKey`.\n"
+          + "Actual value obtained by invoking .toString()");
+
+  public static final ConfigOption<String> KEYGEN_CLASS = ConfigOptions
+      .key("hoodie.datasource.write.keygenerator.class")
+      .stringType()
+      .defaultValue(SimpleAvroKeyGenerator.class.getName())
+      .withDescription("Key generator class, that implements will extract the key out of incoming record");
+
+  public static final ConfigOption<Integer> WRITE_TASK_PARALLELISM = ConfigOptions
+      .key("write.task.parallelism")
+      .intType()
+      .defaultValue(4)
+      .withDescription("Parallelism of tasks that do actual write, default is 4");
+

Review comment:
       We can if we support more index type, currently the new pipeline only supports one. (Although not in this PR, but it is my plan though ~ I don't want to add it then remove ~)




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



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

Posted by GitBox <gi...@apache.org>.
danny0405 commented on a change in pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#discussion_r565016901



##########
File path: hudi-flink/src/main/java/org/apache/hudi/operator/FlinkOptions.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.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");
+
+  // ------------------------------------------------------------------------
+  //  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");

Review comment:
       Replace with `,` instead.




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



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

Posted by GitBox <gi...@apache.org>.
danny0405 commented on a change in pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#discussion_r560731527



##########
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(
+              String.format("Failed to checkpoint Instant %s for source %s",
+                  this.inFlightInstant, this.getClass().getSimpleName()), throwable));
+    }
+  }
+
+  @Override
+  public void notifyCheckpointComplete(long checkpointId) {
+    // start to commit the instant.
+    checkAndCommitWithRetry();
+  }
+
+  @Override
+  public void notifyCheckpointAborted(long checkpointId) {
+    Preconditions.checkState(inFlightCheckpoint == checkpointId,
+        "The aborted checkpoint should always be the last checkpoint");
+    checkAndForceCommit("The last checkpoint was aborted, roll back the last write and throw");
+  }
+
+  @Override
+  public void resetToCheckpoint(long l, @Nullable byte[] checkpointData) throws Exception {
+    if (checkpointData != null) {
+      // restore when any checkpoint completed
+      deserializeCheckpointAndRestore(checkpointData);
+    }
+  }
+
+  @Override
+  public void handleEventFromOperator(int i, OperatorEvent operatorEvent) {
+    // no event to handle
+    Preconditions.checkState(operatorEvent instanceof BatchWriteSuccessEvent,
+        "The coordinator can only handle BatchWriteSuccessEvent");
+    BatchWriteSuccessEvent event = (BatchWriteSuccessEvent) operatorEvent;
+    Preconditions.checkState(event.getInstantTime().equals(this.inFlightInstant),
+        String.format("Receive an unexpected event for instant %s from task %d",
+            event.getInstantTime(), event.getTaskID()));
+    this.eventBuffer[event.getTaskID()] = event;
+  }
+
+  @Override
+  public void subtaskFailed(int i, @Nullable Throwable throwable) {
+    // no operation
+  }
+
+  @Override
+  public void subtaskReset(int i, long l) {
+    // no operation
+  }
+
+  // -------------------------------------------------------------------------
+  //  Utilities
+  // -------------------------------------------------------------------------
+
+  @SuppressWarnings("rawtypes")
+  private void initWriteClient() {
+    writeClient = new HoodieFlinkWriteClient(
+        new HoodieFlinkEngineContext(new FlinkTaskContextSupplier(null)),
+        StreamerUtil.getHoodieClientConfig(this.conf),
+        true);
+  }
+
+  private void initTable() throws IOException {
+    final String basePath = this.conf.getString(FlinkOptions.PATH);
+    final org.apache.hadoop.conf.Configuration hadoopConf = StreamerUtil.getHadoopConf();
+    // Hadoop FileSystem
+    try (FileSystem fs = FSUtils.getFs(basePath, hadoopConf)) {
+      if (!fs.exists(new Path(basePath, HoodieTableMetaClient.METAFOLDER_NAME))) {
+        HoodieTableMetaClient.initTableType(
+            hadoopConf,
+            basePath,
+            HoodieTableType.valueOf(this.conf.getString(FlinkOptions.TABLE_TYPE)),
+            this.conf.getString(FlinkOptions.TABLE_NAME),
+            "archived",
+            this.conf.getString(FlinkOptions.PAYLOAD_CLASS),
+            1);
+        LOG.info("Table initialized");
+      } else {
+        LOG.info("Table [{}/{}] already exists, no need to initialize the table",
+            basePath, this.conf.getString(FlinkOptions.TABLE_NAME));
+      }
+    }
+  }
+
+  static byte[] readBytes(DataInputStream in, int size) throws IOException {
+    byte[] bytes = new byte[size];
+    in.readFully(bytes);
+    return bytes;
+  }
+
+  /**
+   * Serialize the coordinator state. The current implementation may not be super efficient,
+   * but it should not matter that much because most of the state should be rather small.
+   * Large states themselves may already be a problem regardless of how the serialization
+   * is implemented.
+   *
+   * @return A byte array containing the serialized state of the source coordinator.
+   * @throws IOException When something goes wrong in serialization.
+   */
+  private byte[] writeCheckpointBytes() throws IOException {
+    try (ByteArrayOutputStream baos = new ByteArrayOutputStream();
+         DataOutputStream out = new DataOutputViewStreamWrapper(baos)) {
+
+      out.writeLong(this.inFlightCheckpoint);
+      byte[] serializedInstant = this.inFlightInstant.getBytes();
+      out.writeInt(serializedInstant.length);
+      out.write(serializedInstant);
+      out.flush();
+      return baos.toByteArray();
+    }
+  }
+
+  /**
+   * Restore the state of this source coordinator from the state bytes.
+   *
+   * @param bytes The checkpoint bytes that was returned from {@link #writeCheckpointBytes()}
+   * @throws Exception When the deserialization failed.
+   */
+  private void deserializeCheckpointAndRestore(byte[] bytes) throws Exception {
+    try (ByteArrayInputStream bais = new ByteArrayInputStream(bytes);
+         DataInputStream in = new DataInputViewStreamWrapper(bais)) {
+      long checkpointID = in.readLong();
+      int serializedInstantSize = in.readInt();
+      byte[] serializedInstant = readBytes(in, serializedInstantSize);
+      this.inFlightCheckpoint = checkpointID;
+      this.inFlightInstant = new String(serializedInstant);
+    }
+  }
+
+  private void reset() {
+    this.inFlightInstant = "";
+    this.eventBuffer = new BatchWriteSuccessEvent[this.parallelism];
+  }
+
+  private void checkAndForceCommit(String errMsg) {
+    if (!checkReady()) {
+      // forced but still has inflight instant
+      String inflightInstant = writeClient.getInflightAndRequestedInstant(this.conf.getString(FlinkOptions.TABLE_TYPE));
+      if (inflightInstant != null) {
+        assert inflightInstant.equals(this.inFlightInstant);
+        writeClient.rollback(this.inFlightInstant);
+        throw new HoodieException(errMsg);
+      }
+      if (Arrays.stream(eventBuffer).allMatch(Objects::isNull)) {
+        // The last checkpoint finished successfully.
+        return;
+      }
+    }
+    doCommit();
+  }
+
+  private void checkAndCommitWithRetry() {
+    int retryTimes = this.conf.getInteger(FlinkOptions.RETRY_TIMES);
+    long retryIntervalMillis = this.conf.getLong(FlinkOptions.RETRY_INTERVAL_MS);
+    int tryTimes = 0;
+    while (tryTimes++ < retryTimes) {
+      try {
+        if (!checkReady()) {
+          // Do not throw if the try times expires but the event buffer are still not ready,
+          // because we have a force check when next checkpoint starts.
+          waitFor(retryIntervalMillis);
+          continue;
+        }
+        doCommit();
+        return;
+      } catch (Throwable throwable) {
+        String cause = throwable.getCause() == null ? "" : throwable.getCause().toString();
+        LOG.warn("Try to commit the instant {} failed, with times {} and cause {}", this.inFlightInstant, tryTimes, cause);
+        if (tryTimes == retryTimes) {
+          throw new HoodieException(throwable);
+        }
+        waitFor(retryIntervalMillis);
+      }
+    }
+  }
+
+  private void waitFor(long intervalMillis) {
+    try {
+      TimeUnit.MILLISECONDS.sleep(intervalMillis);
+    } catch (InterruptedException e) {
+      LOG.error("Thread interrupted while waiting to retry the instant commits");
+      throw new HoodieException(e);
+    }
+  }
+
+  /** Checks the buffer is ready to commit. */
+  private boolean checkReady() {
+    return Arrays.stream(eventBuffer).allMatch(event ->
+        event != null && event.getInstantTime().equals(this.inFlightInstant));
+  }
+
+  /** Performs the actual commit action. */
+  private void doCommit() {
+    List<WriteStatus> writeResults = Arrays.stream(eventBuffer)
+        .map(BatchWriteSuccessEvent::getWriteStatuses)
+        .flatMap(Collection::stream)
+        .collect(Collectors.toList());
+
+    if (writeResults.size() == 0) {
+      // No data has written, clear the metadata file
+      this.writeClient.deletePendingInstant(this.conf.getString(FlinkOptions.TABLE_TYPE), this.inFlightInstant);
+      reset();
+      return;
+    }
+
+    // commit or rollback
+    long totalErrorRecords = writeResults.stream().map(WriteStatus::getTotalErrorRecords).reduce(Long::sum).orElse(0L);
+    long totalRecords = writeResults.stream().map(WriteStatus::getTotalRecords).reduce(Long::sum).orElse(0L);
+    boolean hasErrors = totalErrorRecords > 0;
+
+    if (!hasErrors || this.conf.getBoolean(FlinkOptions.IGNORE_FAILED_BATCH)) {
+      HashMap<String, String> checkpointCommitMetadata = new HashMap<>();
+      if (hasErrors) {
+        LOG.warn("Some records failed to merge but forcing commit since commitOnErrors set to true. Errors/Total="
+            + totalErrorRecords + "/" + totalRecords);
+      }
+
+      boolean success = writeClient.commit(this.inFlightInstant, writeResults, Option.of(checkpointCommitMetadata));
+      if (success) {
+        reset();
+        LOG.info("Commit instant [{}] success!", this.inFlightInstant);
+      } else {
+        throw new HoodieException(String.format("Commit instant [%s] failed!", this.inFlightInstant));
+      }
+    } else {
+      LOG.error("Error when writing. Errors/Total=" + totalErrorRecords + "/" + totalRecords);
+      LOG.error("The first 100 error messages");
+      writeResults.stream().filter(WriteStatus::hasErrors).limit(100).forEach(ws -> {
+        LOG.error("Global error for partition path {} and fileID {}: {}",
+            ws.getGlobalError(), ws.getPartitionPath(), ws.getFileId());
+        if (ws.getErrors().size() > 0) {
+          ws.getErrors().forEach((key, value) -> LOG.trace("Error for key:" + key + " and value " + value));
+        }
+      });
+      // Rolls back instant
+      writeClient.rollback(this.inFlightInstant);
+      throw new HoodieException(String.format("Commit instant [%s] failed and rolled back !", this.inFlightInstant));
+    }
+  }
+
+  @VisibleForTesting
+  public BatchWriteSuccessEvent[] getEventBuffer() {
+    return eventBuffer;
+  }
+
+  @VisibleForTesting
+  public String getInFlightInstant() {

Review comment:
       > How did the writer know the current instant
   
   Fetch through the `HoodieFlinkWriteClient`, actually the `HoodieTableMetaClient`.




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



[GitHub] [hudi] wangxianghu commented on a change in pull request #2430: [HUDI-1522] Remove the single parallelism operator from the Flink writer

Posted by GitBox <gi...@apache.org>.
wangxianghu commented on a change in pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#discussion_r554857798



##########
File path: hudi-flink/src/main/java/org/apache/hudi/HoodieFlinkStreamer.java
##########
@@ -160,6 +156,19 @@ public static void main(String[] args) throws Exception {
         + "(using the CLI parameter \"--props\") can also be passed command line using this parameter.")
     public List<String> configs = new ArrayList<>();
 
+    @Parameter(names = {"--record-key-field"}, description = "Record key field. Value to be used as the `recordKey` component of `HoodieKey`.\n"
+        + "Actual value will be obtained by invoking .toString() on the field value. Nested fields can be specified using "
+        + "the dot notation eg: `a.b.c`. By default `uuid`.")
+    public String recordKeyField = "uuid";
+
+    @Parameter(names = {"--partition-path-field"}, description = "Partition path field. Value to be used at \n"
+        + "the `partitionPath` component of `HoodieKey`. Actual value obtained by invoking .toString(). By default `partitionpath`.")
+    public String partitionPathField = "partitionpath";
+
+    @Parameter(names = {"--partition-path-field"}, description = "Key generator class, that implements will extract the key out of incoming record.\n"

Review comment:
       little mistake   `--partition-path-field`




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



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

Posted by GitBox <gi...@apache.org>.
yanghua commented on a change in pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#discussion_r561720672



##########
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")
+      .stringType()
+      .defaultValue("ts")
+      .withDescription("Field used in preCombining before actual write. When two records have the same\n"
+          + "key value, we will pick the one with the largest value for the precombine field,\n"
+          + "determined by Object.compareTo(..)");
+
+  public static final ConfigOption<String> PAYLOAD_CLASS = ConfigOptions
+      .key("write.payload.class")
+      .stringType()
+      .defaultValue(OverwriteWithLatestAvroPayload.class.getName())
+      .withDescription("Payload class used. Override this, if you like to roll your own merge logic, when upserting/inserting.\n"
+          + "This will render any value set for the option in-effective");
+
+  /**
+   * Flag to indicate whether to drop duplicates upon insert.
+   * By default insert will accept duplicates, to gain extra performance.
+   */
+  public static final ConfigOption<Boolean> INSERT_DROP_DUPS = ConfigOptions
+      .key("write.insert.drop.duplicates")
+      .booleanType()
+      .defaultValue(false)
+      .withDescription("Flag to indicate whether to drop duplicates upon insert.\n"
+          + "By default insert will accept duplicates, to gain extra performance");
+
+  public static final ConfigOption<Integer> RETRY_TIMES = ConfigOptions
+      .key("write.retry.times")
+      .intType()
+      .defaultValue(3)
+      .withDescription("Flag to indicate how many times streaming job should retry for a failed checkpoint batch.\n"
+          + "By default 3");
+
+  public static final ConfigOption<Long> RETRY_INTERVAL_MS = ConfigOptions
+      .key("write.retry.interval.ms")
+      .longType()
+      .defaultValue(2000L)
+      .withDescription("Flag to indicate how long (by millisecond) before a retry should issued for failed checkpoint batch.\n"
+          + "By default 2000 and it will be doubled by every retry");
+
+  public static final ConfigOption<Boolean> IGNORE_FAILED_BATCH = ConfigOptions
+      .key("write.ignore.failed.batch")
+      .booleanType()
+      .defaultValue(true)
+      .withDescription("Flag to indicate whether to ignore any non exception error (e.g. writestatus error). within a checkpoint batch.\n"
+          + "By default true (in favor of streaming progressing over data integrity)");
+
+  public static final ConfigOption<String> RECORD_KEY_FIELD = ConfigOptions
+      .key(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY)
+      .stringType()
+      .defaultValue("uuid")
+      .withDescription("Record key field. Value to be used as the `recordKey` component of `HoodieKey`.\n"
+          + "Actual value will be obtained by invoking .toString() on the field value. Nested fields can be specified using "
+          + "the dot notation eg: `a.b.c`");
+
+  public static final ConfigOption<String> PARTITION_PATH_FIELD = ConfigOptions
+      .key(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY)
+      .stringType()
+      .defaultValue("partition-path")
+      .withDescription("Partition path field. Value to be used at the `partitionPath` component of `HoodieKey`.\n"
+          + "Actual value obtained by invoking .toString()");
+
+  public static final ConfigOption<String> KEYGEN_CLASS = ConfigOptions
+      .key("hoodie.datasource.write.keygenerator.class")

Review comment:
       any response?




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



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

Posted by GitBox <gi...@apache.org>.
yanghua commented on a change in pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#discussion_r560729693



##########
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")
+      .stringType()
+      .defaultValue("ts")
+      .withDescription("Field used in preCombining before actual write. When two records have the same\n"
+          + "key value, we will pick the one with the largest value for the precombine field,\n"
+          + "determined by Object.compareTo(..)");
+
+  public static final ConfigOption<String> PAYLOAD_CLASS = ConfigOptions
+      .key("write.payload.class")
+      .stringType()
+      .defaultValue(OverwriteWithLatestAvroPayload.class.getName())
+      .withDescription("Payload class used. Override this, if you like to roll your own merge logic, when upserting/inserting.\n"
+          + "This will render any value set for the option in-effective");
+
+  /**
+   * Flag to indicate whether to drop duplicates upon insert.
+   * By default insert will accept duplicates, to gain extra performance.
+   */
+  public static final ConfigOption<Boolean> INSERT_DROP_DUPS = ConfigOptions
+      .key("write.insert.drop.duplicates")
+      .booleanType()
+      .defaultValue(false)
+      .withDescription("Flag to indicate whether to drop duplicates upon insert.\n"
+          + "By default insert will accept duplicates, to gain extra performance");
+
+  public static final ConfigOption<Integer> RETRY_TIMES = ConfigOptions
+      .key("write.retry.times")
+      .intType()
+      .defaultValue(3)
+      .withDescription("Flag to indicate how many times streaming job should retry for a failed checkpoint batch.\n"
+          + "By default 3");
+
+  public static final ConfigOption<Long> RETRY_INTERVAL_MS = ConfigOptions
+      .key("write.retry.interval.ms")
+      .longType()
+      .defaultValue(2000L)
+      .withDescription("Flag to indicate how long (by millisecond) before a retry should issued for failed checkpoint batch.\n"
+          + "By default 2000 and it will be doubled by every retry");
+
+  public static final ConfigOption<Boolean> IGNORE_FAILED_BATCH = ConfigOptions
+      .key("write.ignore.failed.batch")
+      .booleanType()
+      .defaultValue(true)
+      .withDescription("Flag to indicate whether to ignore any non exception error (e.g. writestatus error). within a checkpoint batch.\n"
+          + "By default true (in favor of streaming progressing over data integrity)");
+
+  public static final ConfigOption<String> RECORD_KEY_FIELD = ConfigOptions
+      .key(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY)
+      .stringType()
+      .defaultValue("uuid")
+      .withDescription("Record key field. Value to be used as the `recordKey` component of `HoodieKey`.\n"
+          + "Actual value will be obtained by invoking .toString() on the field value. Nested fields can be specified using "
+          + "the dot notation eg: `a.b.c`");
+
+  public static final ConfigOption<String> PARTITION_PATH_FIELD = ConfigOptions
+      .key(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY)
+      .stringType()
+      .defaultValue("partition-path")
+      .withDescription("Partition path field. Value to be used at the `partitionPath` component of `HoodieKey`.\n"
+          + "Actual value obtained by invoking .toString()");
+
+  public static final ConfigOption<String> KEYGEN_CLASS = ConfigOptions
+      .key("hoodie.datasource.write.keygenerator.class")
+      .stringType()
+      .defaultValue(SimpleAvroKeyGenerator.class.getName())
+      .withDescription("Key generator class, that implements will extract the key out of incoming record");
+
+  public static final ConfigOption<Integer> WRITE_TASK_PARALLELISM = ConfigOptions
+      .key("write.task.parallelism")
+      .intType()
+      .defaultValue(4)
+      .withDescription("Parallelism of tasks that do actual write, default is 4");
+
+  // -------------------------------------------------------------------------
+  //  Utilities
+  // -------------------------------------------------------------------------
+
+  // Remember to update the set when adding new options.
+  public static final List<ConfigOption<?>> OPTIONAL_OPTIONS = Arrays.asList(
+      TABLE_TYPE, OPERATION, PRECOMBINE_FIELD, PAYLOAD_CLASS, INSERT_DROP_DUPS, RETRY_TIMES,
+      RETRY_INTERVAL_MS, IGNORE_FAILED_BATCH, RECORD_KEY_FIELD, PARTITION_PATH_FIELD, KEYGEN_CLASS
+  );
+
+  // Prefix for Hoodie specific properties.
+  public static final String PROPERTIES_PREFIX = "properties.";

Review comment:
       It seems the constant field's boundary of usage is not beyond this class, right? so could it be `private`?




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



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

Posted by GitBox <gi...@apache.org>.
yanghua commented on a change in pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#discussion_r560652915



##########
File path: hudi-flink/src/main/java/org/apache/hudi/schema/FilebasedSchemaProvider.java
##########
@@ -43,16 +45,13 @@
     private static final String TARGET_SCHEMA_FILE_PROP = "hoodie.deltastreamer.schemaprovider.target.schema.file";

Review comment:
       Will we always reuse the config option from `deltastreamer`? Let us think about it with more insight?




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



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

Posted by GitBox <gi...@apache.org>.
yanghua commented on a change in pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#discussion_r560740119



##########
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(FlinkOptions.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(FlinkOptions.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(toHoodieRecord(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(FlinkOptions.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 toHoodieRecord(I record) throws IOException {
+    boolean shouldCombine = this.config.getBoolean(FlinkOptions.INSERT_DROP_DUPS)
+        || WriteOperationType.fromValue(this.config.getString(FlinkOptions.OPERATION)) == WriteOperationType.UPSERT;
+    GenericRecord gr = (GenericRecord) this.converter.convert(this.avroSchema, record);
+    HoodieRecordPayload payload = shouldCombine
+        ? StreamerUtil.createPayload(this.config.getString(FlinkOptions.PAYLOAD_CLASS), gr,
+        (Comparable) HoodieAvroUtils.getNestedFieldVal(gr, this.config.getString(FlinkOptions.PRECOMBINE_FIELD), false))
+        : StreamerUtil.createPayload(this.config.getString(FlinkOptions.PAYLOAD_CLASS), gr);
+    return new HoodieRecord<>(keyGenerator.getKey(gr), payload);
+  }
+
+  private void flushBuffer() {
+    final List<WriteStatus> writeStatus;
+    if (buffer.size() > 0) {
+      writeStatus = writeFunction.apply(buffer, currentInstant);
+      buffer.clear();
+    } else {
+      LOG.info("No data to write in subtask [{}] for instant [{}]", taskID, currentInstant);
+      writeStatus = Collections.emptyList();
+    }
+    this.eventGateway.sendEventToCoordinator(new BatchWriteSuccessEvent(this.taskID, currentInstant, writeStatus));

Review comment:
       Do we need a retry or ack mechanism here? What's the consequence if the coordinator does not receive the event?




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



[GitHub] [hudi] codecov-io edited a comment on pull request #2430: [HUDI-1522] Add a new pipeline for Flink writer

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#issuecomment-757736411


   # [Codecov](https://codecov.io/gh/apache/hudi/pull/2430?src=pr&el=h1) Report
   > Merging [#2430](https://codecov.io/gh/apache/hudi/pull/2430?src=pr&el=desc) (e9f4934) into [master](https://codecov.io/gh/apache/hudi/commit/7ce3ac778eb475bf23ffa31243dc0843ec7d089a?el=desc) (7ce3ac7) will **decrease** coverage by `41.08%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/hudi/pull/2430/graphs/tree.svg?width=650&height=150&src=pr&token=VTTXabwbs2)](https://codecov.io/gh/apache/hudi/pull/2430?src=pr&el=tree)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master   #2430       +/-   ##
   ============================================
   - Coverage     50.76%   9.68%   -41.09%     
   + Complexity     3063      48     -3015     
   ============================================
     Files           419      53      -366     
     Lines         18777    1930    -16847     
     Branches       1918     230     -1688     
   ============================================
   - Hits           9533     187     -9346     
   + Misses         8468    1730     -6738     
   + Partials        776      13      -763     
   ```
   
   | Flag | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | hudicli | `?` | `?` | |
   | hudiclient | `100.00% <ø> (ø)` | `0.00 <ø> (ø)` | |
   | hudicommon | `?` | `?` | |
   | hudiflink | `?` | `?` | |
   | hudihadoopmr | `?` | `?` | |
   | hudisparkdatasource | `?` | `?` | |
   | hudisync | `?` | `?` | |
   | huditimelineservice | `?` | `?` | |
   | hudiutilities | `9.68% <ø> (-59.75%)` | `0.00 <ø> (ø)` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/hudi/pull/2430?src=pr&el=tree) | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | [...va/org/apache/hudi/utilities/IdentitySplitter.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL0lkZW50aXR5U3BsaXR0ZXIuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-2.00%)` | |
   | [...va/org/apache/hudi/utilities/schema/SchemaSet.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NjaGVtYS9TY2hlbWFTZXQuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-3.00%)` | |
   | [...a/org/apache/hudi/utilities/sources/RowSource.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvUm93U291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | [.../org/apache/hudi/utilities/sources/AvroSource.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvQXZyb1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-1.00%)` | |
   | [.../org/apache/hudi/utilities/sources/JsonSource.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvblNvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-1.00%)` | |
   | [...rg/apache/hudi/utilities/sources/CsvDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvQ3N2REZTU291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-10.00%)` | |
   | [...g/apache/hudi/utilities/sources/JsonDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvbkRGU1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | [...apache/hudi/utilities/sources/JsonKafkaSource.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvbkthZmthU291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-6.00%)` | |
   | [...pache/hudi/utilities/sources/ParquetDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvUGFycXVldERGU1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-5.00%)` | |
   | [...lities/schema/SchemaProviderWithPostProcessor.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NjaGVtYS9TY2hlbWFQcm92aWRlcldpdGhQb3N0UHJvY2Vzc29yLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | ... and [386 more](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree-more) | |
   


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



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

Posted by GitBox <gi...@apache.org>.
danny0405 commented on a change in pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#discussion_r559434859



##########
File path: hudi-flink/pom.xml
##########
@@ -124,28 +124,77 @@
       <artifactId>kafka-clients</artifactId>
       <version>${kafka.version}</version>
     </dependency>
+    <dependency>
+      <groupId>org.apache.flink</groupId>
+      <artifactId>flink-hadoop-compatibility_${scala.binary.version}</artifactId>
+      <version>${flink.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.flink</groupId>
+      <artifactId>flink-avro</artifactId>
+      <version>${flink.version}</version>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.flink</groupId>
+      <artifactId>flink-json</artifactId>
+      <version>${flink.version}</version>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.flink</groupId>
+      <artifactId>flink-table-common</artifactId>
+      <version>${flink.version}</version>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.flink</groupId>
+      <artifactId>flink-table-runtime-blink_${scala.binary.version}</artifactId>
+      <version>${flink.version}</version>
+      <scope>provided</scope>
+    </dependency>
 
     <!-- Hadoop -->
     <dependency>
       <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-common</artifactId>
       <scope>compile</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>org.slf4j</groupId>
+          <artifactId>slf4j-log4j12</artifactId>
+        </exclusion>
+      </exclusions>
     </dependency>
     <dependency>
       <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-hdfs</artifactId>
       <scope>compile</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>org.slf4j</groupId>
+          <artifactId>slf4j-log4j12</artifactId>
+        </exclusion>
+      </exclusions>
     </dependency>
     <dependency>
       <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-auth</artifactId>
       <scope>compile</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>org.slf4j</groupId>
+          <artifactId>slf4j-log4j12</artifactId>
+        </exclusion>
+      </exclusions>
     </dependency>
 
     <!-- Avro -->
     <dependency>
       <groupId>org.apache.avro</groupId>
       <artifactId>avro</artifactId>
+      <!-- Override the version to be same with Flink avro -->
+      <version>1.10.0</version>

Review comment:
       No, only Flink needs that ~




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



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

Posted by GitBox <gi...@apache.org>.
danny0405 commented on pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#issuecomment-768022341


   > @danny0405 sorry for the delay on review, I was super busy this week. The bloom index was merged to master, can we add the bloom index option to this PR as well?
   
   I'm not planning to using the BloomFilter index in the new pipeline, instead there is a BloomFilter index backed state index in the following PR, which is more suitable for streaming write.


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



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

Posted by GitBox <gi...@apache.org>.
danny0405 commented on a change in pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#discussion_r565020444



##########
File path: hudi-flink/src/main/java/org/apache/hudi/operator/StreamWriteOperatorCoordinator.java
##########
@@ -0,0 +1,413 @@
+/*
+ * 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(
+              String.format("Failed to checkpoint Instant %s for source %s",
+                  this.inFlightInstant, this.getClass().getSimpleName()), throwable));
+    }
+  }
+
+  @Override
+  public void checkpointComplete(long checkpointId) {
+    // start to commit the instant.
+    checkAndCommitWithRetry();
+  }
+
+  public void notifyCheckpointAborted(long checkpointId) {
+    Preconditions.checkState(inFlightCheckpoint == checkpointId,
+        "The aborted checkpoint should always be the last checkpoint");
+    checkAndForceCommit("The last checkpoint was aborted, roll back the last write and throw");
+  }
+
+  @Override
+  public void resetToCheckpoint(@Nullable byte[] checkpointData) throws Exception {
+    if (checkpointData != null) {
+      // restore when any checkpoint completed
+      deserializeCheckpointAndRestore(checkpointData);
+    }
+  }
+
+  @Override
+  public void handleEventFromOperator(int i, OperatorEvent operatorEvent) {
+    // no event to handle
+    Preconditions.checkState(operatorEvent instanceof BatchWriteSuccessEvent,
+        "The coordinator can only handle BatchWriteSuccessEvent");
+    BatchWriteSuccessEvent event = (BatchWriteSuccessEvent) operatorEvent;
+    Preconditions.checkState(event.getInstantTime().equals(this.inFlightInstant),
+        String.format("Receive an unexpected event for instant %s from task %d",
+            event.getInstantTime(), event.getTaskID()));
+    this.eventBuffer[event.getTaskID()] = event;
+  }
+
+  @Override
+  public void subtaskFailed(int i, @Nullable Throwable throwable) {
+    // no operation
+  }
+
+  // -------------------------------------------------------------------------
+  //  Utilities
+  // -------------------------------------------------------------------------
+
+  @SuppressWarnings("rawtypes")
+  private void initWriteClient() {
+    writeClient = new HoodieFlinkWriteClient(
+        new HoodieFlinkEngineContext(new FlinkTaskContextSupplier(null)),
+        StreamerUtil.getHoodieClientConfig(this.conf),
+        true);
+  }
+
+  private void initTable() throws IOException {
+    final String basePath = this.conf.getString(FlinkOptions.PATH);
+    final org.apache.hadoop.conf.Configuration hadoopConf = StreamerUtil.getHadoopConf();
+    // Hadoop FileSystem
+    try (FileSystem fs = FSUtils.getFs(basePath, hadoopConf)) {
+      if (!fs.exists(new Path(basePath, HoodieTableMetaClient.METAFOLDER_NAME))) {
+        HoodieTableMetaClient.initTableType(
+            hadoopConf,
+            basePath,
+            HoodieTableType.valueOf(this.conf.getString(FlinkOptions.TABLE_TYPE)),
+            this.conf.getString(FlinkOptions.TABLE_NAME),
+            "archived",
+            this.conf.getString(FlinkOptions.PAYLOAD_CLASS),
+            1);
+        LOG.info("Table initialized");
+      } else {
+        LOG.info("Table [{}/{}] already exists, no need to initialize the table",
+            basePath, this.conf.getString(FlinkOptions.TABLE_NAME));
+      }
+    }
+  }
+
+  static byte[] readBytes(DataInputStream in, int size) throws IOException {
+    byte[] bytes = new byte[size];
+    in.readFully(bytes);
+    return bytes;
+  }
+
+  /**
+   * Serialize the coordinator state. The current implementation may not be super efficient,
+   * but it should not matter that much because most of the state should be rather small.
+   * Large states themselves may already be a problem regardless of how the serialization
+   * is implemented.
+   *
+   * @return A byte array containing the serialized state of the source coordinator.
+   * @throws IOException When something goes wrong in serialization.
+   */
+  private byte[] writeCheckpointBytes() throws IOException {
+    try (ByteArrayOutputStream baos = new ByteArrayOutputStream();
+         DataOutputStream out = new DataOutputViewStreamWrapper(baos)) {
+
+      out.writeLong(this.inFlightCheckpoint);
+      byte[] serializedInstant = this.inFlightInstant.getBytes();
+      out.writeInt(serializedInstant.length);
+      out.write(serializedInstant);
+      out.flush();
+      return baos.toByteArray();
+    }
+  }
+
+  /**
+   * Restore the state of this source coordinator from the state bytes.
+   *
+   * @param bytes The checkpoint bytes that was returned from {@link #writeCheckpointBytes()}
+   * @throws Exception When the deserialization failed.
+   */
+  private void deserializeCheckpointAndRestore(byte[] bytes) throws Exception {
+    try (ByteArrayInputStream bais = new ByteArrayInputStream(bytes);
+         DataInputStream in = new DataInputViewStreamWrapper(bais)) {
+      long checkpointID = in.readLong();
+      int serializedInstantSize = in.readInt();
+      byte[] serializedInstant = readBytes(in, serializedInstantSize);
+      this.inFlightCheckpoint = checkpointID;
+      this.inFlightInstant = new String(serializedInstant);
+    }
+  }
+
+  private void reset() {
+    this.inFlightInstant = "";
+    this.eventBuffer = new BatchWriteSuccessEvent[this.parallelism];
+  }
+
+  private void checkAndForceCommit(String errMsg) {
+    if (!checkReady()) {
+      // forced but still has inflight instant
+      String inflightInstant = writeClient.getInflightAndRequestedInstant(this.conf.getString(FlinkOptions.TABLE_TYPE));
+      if (inflightInstant != null) {
+        assert inflightInstant.equals(this.inFlightInstant);
+        writeClient.rollback(this.inFlightInstant);
+        throw new HoodieException(errMsg);
+      }
+      if (Arrays.stream(eventBuffer).allMatch(Objects::isNull)) {
+        // The last checkpoint finished successfully.
+        return;
+      }
+    }
+    doCommit();
+  }
+
+  private void checkAndCommitWithRetry() {
+    int retryTimes = this.conf.getInteger(FlinkOptions.RETRY_TIMES);
+    long retryIntervalMillis = this.conf.getLong(FlinkOptions.RETRY_INTERVAL_MS);
+    int tryTimes = 0;
+    while (tryTimes++ < retryTimes) {
+      try {
+        if (!checkReady()) {
+          // Do not throw if the try times expires but the event buffer are still not ready,
+          // because we have a force check when next checkpoint starts.
+          waitFor(retryIntervalMillis);
+          continue;
+        }
+        doCommit();
+        return;
+      } catch (Throwable throwable) {
+        String cause = throwable.getCause() == null ? "" : throwable.getCause().toString();
+        LOG.warn("Try to commit the instant {} failed, with times {} and cause {}", this.inFlightInstant, tryTimes, cause);
+        if (tryTimes == retryTimes) {
+          throw new HoodieException(throwable);
+        }
+        waitFor(retryIntervalMillis);
+      }
+    }
+  }
+
+  private void waitFor(long intervalMillis) {

Review comment:
       rename to `sleepFor`.




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



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

Posted by GitBox <gi...@apache.org>.
yanghua commented on a change in pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#discussion_r561943377



##########
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:
       Are you really so young?




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



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

Posted by GitBox <gi...@apache.org>.
garyli1019 commented on pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#issuecomment-768353537


   > > @danny0405 sorry for the delay on review, I was super busy this week. The bloom index was merged to master, can we add the bloom index option to this PR as well?
   > 
   > I'm not planning to using the BloomFilter index in the new pipeline, instead there is a BloomFilter index backed state index in the following PR, which is more suitable for streaming write.
   
   @danny0405 yes, using bloom index in a streaming fashion is what we are planning to do next as well. No worry for now, we can try this PR out and continue the work on top of this. Really appreciate your work! This PR LGTM. 


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



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

Posted by GitBox <gi...@apache.org>.
yanghua commented on a change in pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#discussion_r556520897



##########
File path: hudi-flink/src/main/java/org/apache/hudi/util/StreamerUtil.java
##########
@@ -81,16 +103,50 @@ public static DFSPropertiesConfiguration readConfig(FileSystem fs, Path cfgPath,
     return conf;
   }
 
-  public static Configuration getHadoopConf() {
-    return new Configuration();
+  public static org.apache.hadoop.conf.Configuration getHadoopConf() {
+    // create HiveConf from hadoop configuration with hadoop conf directory configured.
+    org.apache.hadoop.conf.Configuration hadoopConf = null;
+    for (String possibleHadoopConfPath : HadoopUtils.possibleHadoopConfPaths(new Configuration())) {

Review comment:
       What I mean is that can we specify a discovery-order, e.g. first find the specified path, and then find a batch of default paths.
   
   In some scenarios where storage is separated from computing, or data synchronization scenarios, users may not necessarily want to load the local "default" configuration.




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



[GitHub] [hudi] codecov-io edited a comment on pull request #2430: [HUDI-1522] Add a new pipeline for Flink writer

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#issuecomment-757736411


   # [Codecov](https://codecov.io/gh/apache/hudi/pull/2430?src=pr&el=h1) Report
   > Merging [#2430](https://codecov.io/gh/apache/hudi/pull/2430?src=pr&el=desc) (c4a04f9) into [master](https://codecov.io/gh/apache/hudi/commit/7ce3ac778eb475bf23ffa31243dc0843ec7d089a?el=desc) (7ce3ac7) will **decrease** coverage by `41.08%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/hudi/pull/2430/graphs/tree.svg?width=650&height=150&src=pr&token=VTTXabwbs2)](https://codecov.io/gh/apache/hudi/pull/2430?src=pr&el=tree)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master   #2430       +/-   ##
   ============================================
   - Coverage     50.76%   9.68%   -41.09%     
   + Complexity     3063      48     -3015     
   ============================================
     Files           419      53      -366     
     Lines         18777    1930    -16847     
     Branches       1918     230     -1688     
   ============================================
   - Hits           9533     187     -9346     
   + Misses         8468    1730     -6738     
   + Partials        776      13      -763     
   ```
   
   | Flag | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | hudicli | `?` | `?` | |
   | hudiclient | `?` | `?` | |
   | hudicommon | `?` | `?` | |
   | hudiflink | `?` | `?` | |
   | hudihadoopmr | `?` | `?` | |
   | hudisparkdatasource | `?` | `?` | |
   | hudisync | `?` | `?` | |
   | huditimelineservice | `?` | `?` | |
   | hudiutilities | `9.68% <ø> (-59.75%)` | `0.00 <ø> (ø)` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/hudi/pull/2430?src=pr&el=tree) | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | [...va/org/apache/hudi/utilities/IdentitySplitter.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL0lkZW50aXR5U3BsaXR0ZXIuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-2.00%)` | |
   | [...va/org/apache/hudi/utilities/schema/SchemaSet.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NjaGVtYS9TY2hlbWFTZXQuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-3.00%)` | |
   | [...a/org/apache/hudi/utilities/sources/RowSource.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvUm93U291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | [.../org/apache/hudi/utilities/sources/AvroSource.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvQXZyb1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-1.00%)` | |
   | [.../org/apache/hudi/utilities/sources/JsonSource.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvblNvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-1.00%)` | |
   | [...rg/apache/hudi/utilities/sources/CsvDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvQ3N2REZTU291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-10.00%)` | |
   | [...g/apache/hudi/utilities/sources/JsonDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvbkRGU1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | [...apache/hudi/utilities/sources/JsonKafkaSource.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvbkthZmthU291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-6.00%)` | |
   | [...pache/hudi/utilities/sources/ParquetDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvUGFycXVldERGU1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-5.00%)` | |
   | [...lities/schema/SchemaProviderWithPostProcessor.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NjaGVtYS9TY2hlbWFQcm92aWRlcldpdGhQb3N0UHJvY2Vzc29yLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | ... and [386 more](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree-more) | |
   


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



[GitHub] [hudi] codecov-io edited a comment on pull request #2430: [HUDI-1522] Add a new pipeline for Flink writer

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#issuecomment-757736411






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



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

Posted by GitBox <gi...@apache.org>.
danny0405 commented on pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#issuecomment-762136567


   Thanks @garyli1019 @yanghua for the review, i have updated based on your comments.


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



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

Posted by GitBox <gi...@apache.org>.
danny0405 commented on a change in pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#discussion_r556376012



##########
File path: hudi-common/src/main/avro/HoodieRollbackMetadata.avsc
##########
@@ -51,10 +51,10 @@
      /* overlaps with 'commitsRollback' field. Adding this to track action type for all the instants being rolled back. */
      {
        "name": "instantsRollback",
-       "default": null,
+       "default": [],
        "type": {
           "type": "array",
-          "default": null,
+          "default": [],
           "items": "HoodieInstantInfo"

Review comment:
       The type for field `instantsRollback` is `array` ( not a union type) which is not nullable, so it is reasonable that the default be an empty array `[]`. The old version Avro does not check the validity.
   
   I also check the usage of method `getInstantsRollback` of class `HoodieRollbackMetadata`, the version less than 1.10 are not affected.




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



[GitHub] [hudi] codecov-io commented on pull request #2430: [HUDI-1522] Remove the single parallelism operator from the Flink writer

Posted by GitBox <gi...@apache.org>.
codecov-io commented on pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#issuecomment-757736411


   # [Codecov](https://codecov.io/gh/apache/hudi/pull/2430?src=pr&el=h1) Report
   > Merging [#2430](https://codecov.io/gh/apache/hudi/pull/2430?src=pr&el=desc) (7961488) into [master](https://codecov.io/gh/apache/hudi/commit/7ce3ac778eb475bf23ffa31243dc0843ec7d089a?el=desc) (7ce3ac7) will **decrease** coverage by `41.08%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/hudi/pull/2430/graphs/tree.svg?width=650&height=150&src=pr&token=VTTXabwbs2)](https://codecov.io/gh/apache/hudi/pull/2430?src=pr&el=tree)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master   #2430       +/-   ##
   ============================================
   - Coverage     50.76%   9.68%   -41.09%     
   + Complexity     3063      48     -3015     
   ============================================
     Files           419      53      -366     
     Lines         18777    1930    -16847     
     Branches       1918     230     -1688     
   ============================================
   - Hits           9533     187     -9346     
   + Misses         8468    1730     -6738     
   + Partials        776      13      -763     
   ```
   
   | Flag | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | hudicli | `?` | `?` | |
   | hudiclient | `?` | `?` | |
   | hudicommon | `?` | `?` | |
   | hudiflink | `?` | `?` | |
   | hudihadoopmr | `?` | `?` | |
   | hudisparkdatasource | `?` | `?` | |
   | hudisync | `?` | `?` | |
   | huditimelineservice | `?` | `?` | |
   | hudiutilities | `9.68% <ø> (-59.75%)` | `0.00 <ø> (ø)` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/hudi/pull/2430?src=pr&el=tree) | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | [...va/org/apache/hudi/utilities/IdentitySplitter.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL0lkZW50aXR5U3BsaXR0ZXIuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-2.00%)` | |
   | [...va/org/apache/hudi/utilities/schema/SchemaSet.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NjaGVtYS9TY2hlbWFTZXQuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-3.00%)` | |
   | [...a/org/apache/hudi/utilities/sources/RowSource.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvUm93U291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | [.../org/apache/hudi/utilities/sources/AvroSource.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvQXZyb1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-1.00%)` | |
   | [.../org/apache/hudi/utilities/sources/JsonSource.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvblNvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-1.00%)` | |
   | [...rg/apache/hudi/utilities/sources/CsvDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvQ3N2REZTU291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-10.00%)` | |
   | [...g/apache/hudi/utilities/sources/JsonDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvbkRGU1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | [...apache/hudi/utilities/sources/JsonKafkaSource.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvbkthZmthU291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-6.00%)` | |
   | [...pache/hudi/utilities/sources/ParquetDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvUGFycXVldERGU1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-5.00%)` | |
   | [...lities/schema/SchemaProviderWithPostProcessor.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NjaGVtYS9TY2hlbWFQcm92aWRlcldpdGhQb3N0UHJvY2Vzc29yLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | ... and [383 more](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree-more) | |
   


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



[GitHub] [hudi] yanghua merged pull request #2430: [HUDI-1522] Add a new pipeline for Flink writer

Posted by GitBox <gi...@apache.org>.
yanghua merged pull request #2430:
URL: https://github.com/apache/hudi/pull/2430


   


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



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

Posted by GitBox <gi...@apache.org>.
yanghua commented on a change in pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#discussion_r557329799



##########
File path: hudi-common/src/main/java/org/apache/hudi/common/model/OverwriteWithLatestAvroPayload.java
##########
@@ -79,6 +79,11 @@ public OverwriteWithLatestAvroPayload preCombine(OverwriteWithLatestAvroPayload
    * @returns {@code true} if record represents a delete record. {@code false} otherwise.
    */
   protected boolean isDeleteRecord(GenericRecord genericRecord) {
+    final String isDeleteKey = "_hoodie_is_deleted";
+    // Modify to be compatible with old version Avro.
+    if (genericRecord.getSchema().getField(isDeleteKey) == null) {
+      return false;
+    }
     Object deleteMarker = genericRecord.get("_hoodie_is_deleted");

Review comment:
       I mean you have defined the `isDeleteKey`, why do not change the literal in this line?




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



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

Posted by GitBox <gi...@apache.org>.
yanghua commented on a change in pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#discussion_r556513381



##########
File path: hudi-flink/src/main/java/org/apache/hudi/streamer/OperationConverter.java
##########
@@ -0,0 +1,32 @@
+/*
+ * 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. */

Review comment:
       What I mean is that can we find a way to move this class into a common place?




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



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

Posted by GitBox <gi...@apache.org>.
yanghua commented on a change in pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#discussion_r559339305



##########
File path: hudi-flink/src/main/java/org/apache/hudi/streamer/Config.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 Config extends Configuration {
+  @Parameter(names = {"--kafka-topic"}, description = "kafka topic", 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)

Review comment:
       There are many minor issues for the description items of these config options, e.g., describe it with a normal sentence(take care of the grammar?), Upper the first letter? In short, keep consistent.




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



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

Posted by GitBox <gi...@apache.org>.
danny0405 commented on a change in pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#discussion_r559431761



##########
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:
       No, this is needed by the `Flink` coordinator.




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



[GitHub] [hudi] codecov-io edited a comment on pull request #2430: [HUDI-1522] Add a new pipeline for Flink writer

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#issuecomment-757736411


   # [Codecov](https://codecov.io/gh/apache/hudi/pull/2430?src=pr&el=h1) Report
   > Merging [#2430](https://codecov.io/gh/apache/hudi/pull/2430?src=pr&el=desc) (fe82597) into [master](https://codecov.io/gh/apache/hudi/commit/7ce3ac778eb475bf23ffa31243dc0843ec7d089a?el=desc) (7ce3ac7) will **decrease** coverage by `41.08%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/hudi/pull/2430/graphs/tree.svg?width=650&height=150&src=pr&token=VTTXabwbs2)](https://codecov.io/gh/apache/hudi/pull/2430?src=pr&el=tree)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master   #2430       +/-   ##
   ============================================
   - Coverage     50.76%   9.68%   -41.09%     
   + Complexity     3063      48     -3015     
   ============================================
     Files           419      53      -366     
     Lines         18777    1930    -16847     
     Branches       1918     230     -1688     
   ============================================
   - Hits           9533     187     -9346     
   + Misses         8468    1730     -6738     
   + Partials        776      13      -763     
   ```
   
   | Flag | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | hudicli | `?` | `?` | |
   | hudiclient | `100.00% <ø> (ø)` | `0.00 <ø> (ø)` | |
   | hudicommon | `?` | `?` | |
   | hudiflink | `?` | `?` | |
   | hudihadoopmr | `?` | `?` | |
   | hudisparkdatasource | `?` | `?` | |
   | hudisync | `?` | `?` | |
   | huditimelineservice | `?` | `?` | |
   | hudiutilities | `9.68% <ø> (-59.75%)` | `0.00 <ø> (ø)` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/hudi/pull/2430?src=pr&el=tree) | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | [...va/org/apache/hudi/utilities/IdentitySplitter.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL0lkZW50aXR5U3BsaXR0ZXIuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-2.00%)` | |
   | [...va/org/apache/hudi/utilities/schema/SchemaSet.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NjaGVtYS9TY2hlbWFTZXQuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-3.00%)` | |
   | [...a/org/apache/hudi/utilities/sources/RowSource.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvUm93U291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | [.../org/apache/hudi/utilities/sources/AvroSource.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvQXZyb1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-1.00%)` | |
   | [.../org/apache/hudi/utilities/sources/JsonSource.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvblNvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-1.00%)` | |
   | [...rg/apache/hudi/utilities/sources/CsvDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvQ3N2REZTU291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-10.00%)` | |
   | [...g/apache/hudi/utilities/sources/JsonDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvbkRGU1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | [...apache/hudi/utilities/sources/JsonKafkaSource.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvbkthZmthU291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-6.00%)` | |
   | [...pache/hudi/utilities/sources/ParquetDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvUGFycXVldERGU1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-5.00%)` | |
   | [...lities/schema/SchemaProviderWithPostProcessor.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NjaGVtYS9TY2hlbWFQcm92aWRlcldpdGhQb3N0UHJvY2Vzc29yLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | ... and [386 more](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree-more) | |
   


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



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

Posted by GitBox <gi...@apache.org>.
yanghua commented on a change in pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#discussion_r556336937



##########
File path: hudi-common/src/main/java/org/apache/hudi/common/model/OverwriteWithLatestAvroPayload.java
##########
@@ -79,6 +79,11 @@ public OverwriteWithLatestAvroPayload preCombine(OverwriteWithLatestAvroPayload
    * @returns {@code true} if record represents a delete record. {@code false} otherwise.
    */
   protected boolean isDeleteRecord(GenericRecord genericRecord) {
+    final String isDeleteKey = "_hoodie_is_deleted";
+    // Modify to be compatible with old version Avro.
+    if (genericRecord.getSchema().getField(isDeleteKey) == null) {
+      return false;
+    }
     Object deleteMarker = genericRecord.get("_hoodie_is_deleted");

Review comment:
       Why do not change the literal to the variable that you defined?




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



[GitHub] [hudi] codecov-io edited a comment on pull request #2430: [HUDI-1522] Add a new pipeline for Flink writer

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#issuecomment-757736411


   # [Codecov](https://codecov.io/gh/apache/hudi/pull/2430?src=pr&el=h1) Report
   > Merging [#2430](https://codecov.io/gh/apache/hudi/pull/2430?src=pr&el=desc) (3e3ef16) into [master](https://codecov.io/gh/apache/hudi/commit/7ce3ac778eb475bf23ffa31243dc0843ec7d089a?el=desc) (7ce3ac7) will **decrease** coverage by `41.08%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/hudi/pull/2430/graphs/tree.svg?width=650&height=150&src=pr&token=VTTXabwbs2)](https://codecov.io/gh/apache/hudi/pull/2430?src=pr&el=tree)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master   #2430       +/-   ##
   ============================================
   - Coverage     50.76%   9.68%   -41.09%     
   + Complexity     3063      48     -3015     
   ============================================
     Files           419      53      -366     
     Lines         18777    1930    -16847     
     Branches       1918     230     -1688     
   ============================================
   - Hits           9533     187     -9346     
   + Misses         8468    1730     -6738     
   + Partials        776      13      -763     
   ```
   
   | Flag | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | hudicli | `?` | `?` | |
   | hudiclient | `100.00% <ø> (ø)` | `0.00 <ø> (ø)` | |
   | hudicommon | `?` | `?` | |
   | hudiflink | `?` | `?` | |
   | hudihadoopmr | `?` | `?` | |
   | hudisparkdatasource | `?` | `?` | |
   | hudisync | `?` | `?` | |
   | huditimelineservice | `?` | `?` | |
   | hudiutilities | `9.68% <ø> (-59.75%)` | `0.00 <ø> (ø)` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/hudi/pull/2430?src=pr&el=tree) | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | [...va/org/apache/hudi/utilities/IdentitySplitter.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL0lkZW50aXR5U3BsaXR0ZXIuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-2.00%)` | |
   | [...va/org/apache/hudi/utilities/schema/SchemaSet.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NjaGVtYS9TY2hlbWFTZXQuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-3.00%)` | |
   | [...a/org/apache/hudi/utilities/sources/RowSource.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvUm93U291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | [.../org/apache/hudi/utilities/sources/AvroSource.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvQXZyb1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-1.00%)` | |
   | [.../org/apache/hudi/utilities/sources/JsonSource.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvblNvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-1.00%)` | |
   | [...rg/apache/hudi/utilities/sources/CsvDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvQ3N2REZTU291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-10.00%)` | |
   | [...g/apache/hudi/utilities/sources/JsonDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvbkRGU1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | [...apache/hudi/utilities/sources/JsonKafkaSource.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvbkthZmthU291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-6.00%)` | |
   | [...pache/hudi/utilities/sources/ParquetDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvUGFycXVldERGU1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-5.00%)` | |
   | [...lities/schema/SchemaProviderWithPostProcessor.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NjaGVtYS9TY2hlbWFQcm92aWRlcldpdGhQb3N0UHJvY2Vzc29yLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | ... and [386 more](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree-more) | |
   


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



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

Posted by GitBox <gi...@apache.org>.
danny0405 commented on a change in pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#discussion_r562371379



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




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



[GitHub] [hudi] danny0405 commented on a change in pull request #2430: [HUDI-1522] Remove the single parallelism operator from the Flink writer

Posted by GitBox <gi...@apache.org>.
danny0405 commented on a change in pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#discussion_r554842872



##########
File path: hudi-common/src/main/avro/HoodieRollbackMetadata.avsc
##########
@@ -51,10 +51,10 @@
      /* overlaps with 'commitsRollback' field. Adding this to track action type for all the instants being rolled back. */
      {
        "name": "instantsRollback",
-       "default": null,
+       "default": [],
        "type": {
           "type": "array",
-          "default": null,
+          "default": [],
           "items": "HoodieInstantInfo"

Review comment:
       This change is used to adapter Avro version 1.10+, which is need for Flink writer.




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



[GitHub] [hudi] danny0405 commented on a change in pull request #2430: [HUDI-1522] Remove the single parallelism operator from the Flink writer

Posted by GitBox <gi...@apache.org>.
danny0405 commented on a change in pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#discussion_r554904669



##########
File path: hudi-flink/src/main/java/org/apache/hudi/HoodieFlinkStreamer.java
##########
@@ -160,6 +156,19 @@ public static void main(String[] args) throws Exception {
         + "(using the CLI parameter \"--props\") can also be passed command line using this parameter.")
     public List<String> configs = new ArrayList<>();
 
+    @Parameter(names = {"--record-key-field"}, description = "Record key field. Value to be used as the `recordKey` component of `HoodieKey`.\n"
+        + "Actual value will be obtained by invoking .toString() on the field value. Nested fields can be specified using "
+        + "the dot notation eg: `a.b.c`. By default `uuid`.")
+    public String recordKeyField = "uuid";
+
+    @Parameter(names = {"--partition-path-field"}, description = "Partition path field. Value to be used at \n"
+        + "the `partitionPath` component of `HoodieKey`. Actual value obtained by invoking .toString(). By default `partitionpath`.")
+    public String partitionPathField = "partitionpath";
+
+    @Parameter(names = {"--partition-path-field"}, description = "Key generator class, that implements will extract the key out of incoming record.\n"

Review comment:
       Oops, thanks for the reminder.




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



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

Posted by GitBox <gi...@apache.org>.
danny0405 commented on a change in pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#discussion_r556396251



##########
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:
       Personally i prefer cleaner and shorter names, because options are phrase of Flink connectors, and it is under the package `org.apache.hudi.operator`, we can make it more specific if we introduce other config options class.




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



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

Posted by GitBox <gi...@apache.org>.
danny0405 commented on a change in pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#discussion_r562365771



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




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



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

Posted by GitBox <gi...@apache.org>.
yanghua commented on a change in pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#discussion_r561953785



##########
File path: hudi-flink/src/test/resources/test_read_schema.avsc
##########
@@ -0,0 +1,45 @@
+/*
+ * 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.
+ */
+{
+  "type" : "record",
+  "name" : "record",
+  "fields" : [ {

Review comment:
       Can we follow the same style about the test schema, align with the existed file, e.g. `HoodieCleanerPlan.avsc`




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



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

Posted by GitBox <gi...@apache.org>.
yanghua commented on a change in pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#discussion_r560727023



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

Review comment:
       Did you check if this config is case sensitive? That's to say if it works fine for `merge_on_read`.




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



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

Posted by GitBox <gi...@apache.org>.
danny0405 commented on a change in pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#discussion_r558063116



##########
File path: hudi-flink/src/main/java/org/apache/hudi/util/StreamerUtil.java
##########
@@ -81,16 +103,50 @@ public static DFSPropertiesConfiguration readConfig(FileSystem fs, Path cfgPath,
     return conf;
   }
 
-  public static Configuration getHadoopConf() {
-    return new Configuration();
+  public static org.apache.hadoop.conf.Configuration getHadoopConf() {
+    // create HiveConf from hadoop configuration with hadoop conf directory configured.
+    org.apache.hadoop.conf.Configuration hadoopConf = null;
+    for (String possibleHadoopConfPath : HadoopUtils.possibleHadoopConfPaths(new Configuration())) {

Review comment:
       I think it is reasonable that the explicit specified path has the highest priority.




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



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

Posted by GitBox <gi...@apache.org>.
danny0405 commented on a change in pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#discussion_r559432710



##########
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:
       The class is used by the `HoodieFlinkStreamer`.




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



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

Posted by GitBox <gi...@apache.org>.
danny0405 commented on a change in pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#discussion_r558106065



##########
File path: hudi-flink/pom.xml
##########
@@ -124,28 +124,77 @@
       <artifactId>kafka-clients</artifactId>
       <version>${kafka.version}</version>
     </dependency>
+    <dependency>
+      <groupId>org.apache.flink</groupId>
+      <artifactId>flink-hadoop-compatibility_${scala.binary.version}</artifactId>
+      <version>${flink.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.flink</groupId>
+      <artifactId>flink-avro</artifactId>
+      <version>${flink.version}</version>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.flink</groupId>
+      <artifactId>flink-json</artifactId>
+      <version>${flink.version}</version>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.flink</groupId>
+      <artifactId>flink-table-common</artifactId>
+      <version>${flink.version}</version>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.flink</groupId>
+      <artifactId>flink-table-runtime-blink_${scala.binary.version}</artifactId>

Review comment:
       I think we should not package the in-compatible Flink jars into the bundle, just like the guideline [1], the hudi bundle is used as a spark shell library.
   
   The table jars introduced here like `flink-table-common` and `flink-table-runtime-blink` are only used by the new pipeline, so we can package the into the `hoodie-flink` jar, the old pipeline should not be affected. The data stream api should be compatible itself, if the old pipeline does not use API comes from 1.10+ version.
   
   [1] https://hudi.apache.org/docs/quick-start-guide.html




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



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

Posted by GitBox <gi...@apache.org>.
danny0405 commented on a change in pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#discussion_r557012704



##########
File path: hudi-flink/src/main/java/org/apache/hudi/util/StreamerUtil.java
##########
@@ -81,16 +103,50 @@ public static DFSPropertiesConfiguration readConfig(FileSystem fs, Path cfgPath,
     return conf;
   }
 
-  public static Configuration getHadoopConf() {
-    return new Configuration();
+  public static org.apache.hadoop.conf.Configuration getHadoopConf() {
+    // create HiveConf from hadoop configuration with hadoop conf directory configured.
+    org.apache.hadoop.conf.Configuration hadoopConf = null;
+    for (String possibleHadoopConfPath : HadoopUtils.possibleHadoopConfPaths(new Configuration())) {

Review comment:
       The method firstly find the specified path `fs.hdfs.hadoopconf`, then directory `HADOOP_CONF_DIR` `HADOOP_HOME/conf` `HADOOP_HOME/etc/hadoop` from the system environment.
   
   Even if storage is separated from computing, the `FileSystem` we created is still correct, if we split the hadoop conf files correctly.
   
   In any case, we should pass an empty hadoop configuration.




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



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

Posted by GitBox <gi...@apache.org>.
danny0405 commented on a change in pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#discussion_r556387652



##########
File path: hudi-flink/src/main/java/org/apache/hudi/util/StreamerUtil.java
##########
@@ -122,21 +193,62 @@ public static HoodieRecordPayload createPayload(String payloadClass, GenericReco
     }
   }
 
-  public static HoodieWriteConfig getHoodieClientConfig(HoodieFlinkStreamer.Config cfg) {
-    FileSystem fs = FSUtils.getFs(cfg.targetBasePath, getHadoopConf());
+  /**
+   * Create a payload class via reflection, do not ordering/precombine value.
+   */
+  public static HoodieRecordPayload createPayload(String payloadClass, GenericRecord record)
+      throws IOException {
+    try {
+      return (HoodieRecordPayload) ReflectionUtils.loadClass(payloadClass,
+          new Class<?>[] {Option.class}, Option.of(record));
+    } catch (Throwable e) {
+      throw new IOException("Could not create payload for class: " + payloadClass, e);
+    }
+  }
+
+  public static HoodieWriteConfig getHoodieClientConfig(Configuration conf) {
     HoodieWriteConfig.Builder builder =
-        HoodieWriteConfig.newBuilder().withEngineType(EngineType.FLINK).withPath(cfg.targetBasePath).combineInput(cfg.filterDupes, true)
-            .withCompactionConfig(HoodieCompactionConfig.newBuilder().withPayloadClass(cfg.payloadClassName).build())
-            .withPayloadConfig(HoodiePayloadConfig.newBuilder().withPayloadOrderingField(cfg.sourceOrderingField)
-                .build())
-            .forTable(cfg.targetTableName)
-            .withAutoCommit(false)
-            .withProps(readConfig(fs, new Path(cfg.propsFilePath), cfg.configs)
-                .getConfig());
-
-    builder = builder.withSchema(new FilebasedSchemaProvider(getProps(cfg)).getTargetSchema().toString());
-    HoodieWriteConfig config = builder.build();
-    return config;
+            HoodieWriteConfig.newBuilder()
+                    .withEngineType(EngineType.FLINK)
+                    .withPath(conf.getString(HoodieOptions.PATH))
+                    .combineInput(conf.getBoolean(HoodieOptions.INSERT_DROP_DUPS), true)
+                    .withCompactionConfig(
+                        HoodieCompactionConfig.newBuilder()
+                            .withPayloadClass(conf.getString(HoodieOptions.PAYLOAD_CLASS))
+                            .build())
+                    .forTable(conf.getString(HoodieOptions.TABLE_NAME))
+                    .withAutoCommit(false)
+                    .withProps(flinkConf2TypedProperties(HoodieOptions.flatOptions(conf)));
+
+    builder = builder.withSchema(getSourceSchema(conf).toString());
+    return builder.build();
+  }
+
+  /**
+   * Converts the give {@link Configuration} to {@link TypedProperties}.
+   * The default values are also set up.
+   *
+   * @param conf The flink configuration
+   * @return a TypedProperties instance
+   */
+  public static TypedProperties flinkConf2TypedProperties(Configuration conf) {
+    Properties properties = new Properties();
+    // put all the set up options
+    conf.addAllToProperties(properties);
+    // put all the default options
+    for (ConfigOption<?> option : HoodieOptions.OPTIONAL_OPTIONS) {
+      if (!conf.contains(option)) {
+        properties.put(option.key(), option.defaultValue());
+      }
+    }
+    return new TypedProperties(properties);
   }
 
+  public static void checkRequiredProperties(TypedProperties props, List<String> checkPropNames) {
+    checkPropNames.forEach(prop -> {
+      if (!props.containsKey(prop)) {
+        throw new HoodieNotSupportedException("Required property " + prop + " is missing");

Review comment:
       I just copy it from the old code, use `Preconditions.checkState` instead.




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



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

Posted by GitBox <gi...@apache.org>.
danny0405 commented on a change in pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#discussion_r559430887



##########
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:
       To test the data buffer we received, the statistics in `WriteStatus` are all about written data which is different. 




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



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

Posted by GitBox <gi...@apache.org>.
yanghua commented on a change in pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#discussion_r561719568



##########
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:
       > The config options in HoodieWriteConfig are too verbose. 
   
   That's true.
   
   But for the same configuration, if we have multiple pointers to it, it will bring the risk of losing control. In the future, if we have some changes to this configuration, such as making it depreciated. We may lose our unified control over it.




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



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

Posted by GitBox <gi...@apache.org>.
danny0405 commented on a change in pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#discussion_r556391388



##########
File path: hudi-common/src/main/java/org/apache/hudi/avro/HoodieAvroUtils.java
##########
@@ -217,7 +217,7 @@ public static String addMetadataColumnTypes(String hiveColumnTypes) {
 
   private static Schema initRecordKeySchema() {
     Schema.Field recordKeyField =
-        new Schema.Field(HoodieRecord.RECORD_KEY_METADATA_FIELD, METADATA_FIELD_SCHEMA, "", NullNode.getInstance());

Review comment:
       The old usage is depredated, has checked the other usages and switch to `null`s.




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



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

Posted by GitBox <gi...@apache.org>.
danny0405 commented on a change in pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#discussion_r556394468



##########
File path: hudi-flink/pom.xml
##########
@@ -124,28 +124,77 @@
       <artifactId>kafka-clients</artifactId>
       <version>${kafka.version}</version>
     </dependency>
+    <dependency>
+      <groupId>org.apache.flink</groupId>
+      <artifactId>flink-hadoop-compatibility_${scala.binary.version}</artifactId>
+      <version>${flink.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.flink</groupId>
+      <artifactId>flink-avro</artifactId>
+      <version>${flink.version}</version>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.flink</groupId>
+      <artifactId>flink-json</artifactId>
+      <version>${flink.version}</version>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.flink</groupId>
+      <artifactId>flink-table-common</artifactId>
+      <version>${flink.version}</version>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.flink</groupId>
+      <artifactId>flink-table-runtime-blink_${scala.binary.version}</artifactId>

Review comment:
       That's why i mark the dependency as provided, so that the artifacts are not packaged into the final release, low-level flink version users should not be affected.




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



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

Posted by GitBox <gi...@apache.org>.
yanghua commented on a change in pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#discussion_r562465971



##########
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:
       sounds good




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



[GitHub] [hudi] codecov-io edited a comment on pull request #2430: [HUDI-1522] Add a new pipeline for Flink writer

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#issuecomment-757736411


   # [Codecov](https://codecov.io/gh/apache/hudi/pull/2430?src=pr&el=h1) Report
   > Merging [#2430](https://codecov.io/gh/apache/hudi/pull/2430?src=pr&el=desc) (9324312) into [master](https://codecov.io/gh/apache/hudi/commit/a38612b10f6ae04644519270f9b5eb631a77c148?el=desc) (a38612b) will **decrease** coverage by `41.00%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/hudi/pull/2430/graphs/tree.svg?width=650&height=150&src=pr&token=VTTXabwbs2)](https://codecov.io/gh/apache/hudi/pull/2430?src=pr&el=tree)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master   #2430       +/-   ##
   ============================================
   - Coverage     50.69%   9.68%   -41.01%     
   + Complexity     3059      48     -3011     
   ============================================
     Files           419      53      -366     
     Lines         18810    1930    -16880     
     Branches       1924     230     -1694     
   ============================================
   - Hits           9535     187     -9348     
   + Misses         8498    1730     -6768     
   + Partials        777      13      -764     
   ```
   
   | Flag | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | hudicli | `?` | `?` | |
   | hudiclient | `100.00% <ø> (ø)` | `0.00 <ø> (ø)` | |
   | hudicommon | `?` | `?` | |
   | hudiflink | `?` | `?` | |
   | hudihadoopmr | `?` | `?` | |
   | hudisparkdatasource | `?` | `?` | |
   | hudisync | `?` | `?` | |
   | huditimelineservice | `?` | `?` | |
   | hudiutilities | `9.68% <ø> (-59.80%)` | `0.00 <ø> (ø)` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/hudi/pull/2430?src=pr&el=tree) | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | [...va/org/apache/hudi/utilities/IdentitySplitter.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL0lkZW50aXR5U3BsaXR0ZXIuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-2.00%)` | |
   | [...va/org/apache/hudi/utilities/schema/SchemaSet.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NjaGVtYS9TY2hlbWFTZXQuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-3.00%)` | |
   | [...a/org/apache/hudi/utilities/sources/RowSource.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvUm93U291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | [.../org/apache/hudi/utilities/sources/AvroSource.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvQXZyb1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-1.00%)` | |
   | [.../org/apache/hudi/utilities/sources/JsonSource.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvblNvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-1.00%)` | |
   | [...rg/apache/hudi/utilities/sources/CsvDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvQ3N2REZTU291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-10.00%)` | |
   | [...g/apache/hudi/utilities/sources/JsonDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvbkRGU1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | [...apache/hudi/utilities/sources/JsonKafkaSource.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvbkthZmthU291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-6.00%)` | |
   | [...pache/hudi/utilities/sources/ParquetDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvUGFycXVldERGU1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-5.00%)` | |
   | [...lities/schema/SchemaProviderWithPostProcessor.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NjaGVtYS9TY2hlbWFQcm92aWRlcldpdGhQb3N0UHJvY2Vzc29yLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | ... and [386 more](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree-more) | |
   


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



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

Posted by GitBox <gi...@apache.org>.
danny0405 commented on a change in pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#discussion_r558106065



##########
File path: hudi-flink/pom.xml
##########
@@ -124,28 +124,77 @@
       <artifactId>kafka-clients</artifactId>
       <version>${kafka.version}</version>
     </dependency>
+    <dependency>
+      <groupId>org.apache.flink</groupId>
+      <artifactId>flink-hadoop-compatibility_${scala.binary.version}</artifactId>
+      <version>${flink.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.flink</groupId>
+      <artifactId>flink-avro</artifactId>
+      <version>${flink.version}</version>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.flink</groupId>
+      <artifactId>flink-json</artifactId>
+      <version>${flink.version}</version>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.flink</groupId>
+      <artifactId>flink-table-common</artifactId>
+      <version>${flink.version}</version>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.flink</groupId>
+      <artifactId>flink-table-runtime-blink_${scala.binary.version}</artifactId>

Review comment:
       I think we should not package the in-compatible Flink jars into the bundle, just like the guideline [1], the hudi bundle is used as a spark shell library. Here only `spark-sql` jar are packaged into Hoodie jars.
   
   The table jars introduced here like `flink-table-common` and `flink-table-runtime-blink` are only used by the new pipeline, so we can package the into the `hoodie-flink` jar, the old pipeline should not be affected. The data stream api should be compatible itself, if the old pipeline does not use API comes from 1.10+ version.
   
   [1] https://hudi.apache.org/docs/quick-start-guide.html




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



[GitHub] [hudi] codecov-io edited a comment on pull request #2430: [HUDI-1522] Add a new pipeline for Flink writer

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#issuecomment-757736411


   # [Codecov](https://codecov.io/gh/apache/hudi/pull/2430?src=pr&el=h1) Report
   > Merging [#2430](https://codecov.io/gh/apache/hudi/pull/2430?src=pr&el=desc) (2c4fa32) into [master](https://codecov.io/gh/apache/hudi/commit/c4afd179c1983a382b8a5197d800b0f5dba254de?el=desc) (c4afd17) will **increase** coverage by `2.21%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/hudi/pull/2430/graphs/tree.svg?width=650&height=150&src=pr&token=VTTXabwbs2)](https://codecov.io/gh/apache/hudi/pull/2430?src=pr&el=tree)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #2430      +/-   ##
   ============================================
   + Coverage     50.18%   52.39%   +2.21%     
   + Complexity     3050      552    -2498     
   ============================================
     Files           419       92     -327     
     Lines         18931     4096   -14835     
     Branches       1948      480    -1468     
   ============================================
   - Hits           9500     2146    -7354     
   + Misses         8656     1751    -6905     
   + Partials        775      199     -576     
   ```
   
   | Flag | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | hudicli | `37.21% <ø> (ø)` | `0.00 <ø> (ø)` | |
   | hudiclient | `100.00% <ø> (ø)` | `0.00 <ø> (ø)` | |
   | hudicommon | `?` | `?` | |
   | hudiflink | `?` | `?` | |
   | hudihadoopmr | `?` | `?` | |
   | hudisparkdatasource | `?` | `?` | |
   | hudisync | `?` | `?` | |
   | huditimelineservice | `?` | `?` | |
   | hudiutilities | `69.43% <ø> (ø)` | `0.00 <ø> (ø)` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/hudi/pull/2430?src=pr&el=tree) | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | [...udi/common/table/timeline/dto/CompactionOpDTO.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL3RhYmxlL3RpbWVsaW5lL2R0by9Db21wYWN0aW9uT3BEVE8uamF2YQ==) | | | |
   | [...3/internal/HoodieBulkInsertDataInternalWriter.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS1zcGFyay1kYXRhc291cmNlL2h1ZGktc3BhcmszL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9odWRpL3NwYXJrMy9pbnRlcm5hbC9Ib29kaWVCdWxrSW5zZXJ0RGF0YUludGVybmFsV3JpdGVyLmphdmE=) | | | |
   | [...i/common/table/timeline/HoodieDefaultTimeline.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL3RhYmxlL3RpbWVsaW5lL0hvb2RpZURlZmF1bHRUaW1lbGluZS5qYXZh) | | | |
   | [.../java/org/apache/hudi/common/util/HoodieTimer.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL3V0aWwvSG9vZGllVGltZXIuamF2YQ==) | | | |
   | [...able/timeline/versioning/AbstractMigratorBase.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL3RhYmxlL3RpbWVsaW5lL3ZlcnNpb25pbmcvQWJzdHJhY3RNaWdyYXRvckJhc2UuamF2YQ==) | | | |
   | [...common/table/log/HoodieMergedLogRecordScanner.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL3RhYmxlL2xvZy9Ib29kaWVNZXJnZWRMb2dSZWNvcmRTY2FubmVyLmphdmE=) | | | |
   | [...i/bootstrap/SparkParquetBootstrapDataProvider.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS1zcGFyay1kYXRhc291cmNlL2h1ZGktc3Bhcmsvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvYm9vdHN0cmFwL1NwYXJrUGFycXVldEJvb3RzdHJhcERhdGFQcm92aWRlci5qYXZh) | | | |
   | [...rg/apache/hudi/hadoop/HoodieROTablePathFilter.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS1oYWRvb3AtbXIvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvaGFkb29wL0hvb2RpZVJPVGFibGVQYXRoRmlsdGVyLmphdmE=) | | | |
   | [...rg/apache/hudi/common/model/HoodieFileGroupId.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL21vZGVsL0hvb2RpZUZpbGVHcm91cElkLmphdmE=) | | | |
   | [...pache/hudi/hadoop/HoodieColumnProjectionUtils.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS1oYWRvb3AtbXIvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvaGFkb29wL0hvb2RpZUNvbHVtblByb2plY3Rpb25VdGlscy5qYXZh) | | | |
   | ... and [307 more](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree-more) | |
   


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



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

Posted by GitBox <gi...@apache.org>.
danny0405 commented on a change in pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#discussion_r559439930



##########
File path: hudi-flink/src/main/java/org/apache/hudi/streamer/Config.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 Config extends Configuration {
+  @Parameter(names = {"--kafka-topic"}, description = "kafka topic", 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)

Review comment:
       Okey




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



[GitHub] [hudi] codecov-io edited a comment on pull request #2430: [HUDI-1522] Add a new pipeline for Flink writer

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#issuecomment-757736411


   # [Codecov](https://codecov.io/gh/apache/hudi/pull/2430?src=pr&el=h1) Report
   > Merging [#2430](https://codecov.io/gh/apache/hudi/pull/2430?src=pr&el=desc) (7c96c0c) into [master](https://codecov.io/gh/apache/hudi/commit/7ce3ac778eb475bf23ffa31243dc0843ec7d089a?el=desc) (7ce3ac7) will **decrease** coverage by `41.08%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/hudi/pull/2430/graphs/tree.svg?width=650&height=150&src=pr&token=VTTXabwbs2)](https://codecov.io/gh/apache/hudi/pull/2430?src=pr&el=tree)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master   #2430       +/-   ##
   ============================================
   - Coverage     50.76%   9.68%   -41.09%     
   + Complexity     3063      48     -3015     
   ============================================
     Files           419      53      -366     
     Lines         18777    1930    -16847     
     Branches       1918     230     -1688     
   ============================================
   - Hits           9533     187     -9346     
   + Misses         8468    1730     -6738     
   + Partials        776      13      -763     
   ```
   
   | Flag | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | hudicli | `?` | `?` | |
   | hudiclient | `?` | `?` | |
   | hudicommon | `?` | `?` | |
   | hudiflink | `?` | `?` | |
   | hudihadoopmr | `?` | `?` | |
   | hudisparkdatasource | `?` | `?` | |
   | hudisync | `?` | `?` | |
   | huditimelineservice | `?` | `?` | |
   | hudiutilities | `9.68% <ø> (-59.75%)` | `0.00 <ø> (ø)` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/hudi/pull/2430?src=pr&el=tree) | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | [...va/org/apache/hudi/utilities/IdentitySplitter.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL0lkZW50aXR5U3BsaXR0ZXIuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-2.00%)` | |
   | [...va/org/apache/hudi/utilities/schema/SchemaSet.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NjaGVtYS9TY2hlbWFTZXQuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-3.00%)` | |
   | [...a/org/apache/hudi/utilities/sources/RowSource.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvUm93U291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | [.../org/apache/hudi/utilities/sources/AvroSource.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvQXZyb1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-1.00%)` | |
   | [.../org/apache/hudi/utilities/sources/JsonSource.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvblNvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-1.00%)` | |
   | [...rg/apache/hudi/utilities/sources/CsvDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvQ3N2REZTU291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-10.00%)` | |
   | [...g/apache/hudi/utilities/sources/JsonDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvbkRGU1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | [...apache/hudi/utilities/sources/JsonKafkaSource.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvbkthZmthU291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-6.00%)` | |
   | [...pache/hudi/utilities/sources/ParquetDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvUGFycXVldERGU1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-5.00%)` | |
   | [...lities/schema/SchemaProviderWithPostProcessor.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NjaGVtYS9TY2hlbWFQcm92aWRlcldpdGhQb3N0UHJvY2Vzc29yLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | ... and [386 more](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree-more) | |
   


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



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

Posted by GitBox <gi...@apache.org>.
danny0405 commented on a change in pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#discussion_r560667902



##########
File path: hudi-flink/src/main/java/org/apache/hudi/schema/FilebasedSchemaProvider.java
##########
@@ -43,16 +45,13 @@
     private static final String TARGET_SCHEMA_FILE_PROP = "hoodie.deltastreamer.schemaprovider.target.schema.file";

Review comment:
       We should keep the option for compatibility.




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



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

Posted by GitBox <gi...@apache.org>.
danny0405 commented on a change in pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#discussion_r560706743



##########
File path: hudi-flink/src/main/java/org/apache/hudi/schema/FilebasedSchemaProvider.java
##########
@@ -43,16 +45,13 @@
     private static final String TARGET_SCHEMA_FILE_PROP = "hoodie.deltastreamer.schemaprovider.target.schema.file";

Review comment:
       Sure, see https://issues.apache.org/jira/browse/HUDI-1541




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



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

Posted by GitBox <gi...@apache.org>.
danny0405 commented on a change in pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#discussion_r562366032



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




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



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

Posted by GitBox <gi...@apache.org>.
danny0405 commented on a change in pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#discussion_r565019061



##########
File path: hudi-flink/src/main/java/org/apache/hudi/operator/FlinkOptions.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.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");
+
+  // ------------------------------------------------------------------------
+  //  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")
+      .stringType()
+      .defaultValue("ts")
+      .withDescription("Field used in preCombining before actual write. When two records have the same\n"
+          + "key value, we will pick the one with the largest value for the precombine field,\n"
+          + "determined by Object.compareTo(..)");
+
+  public static final ConfigOption<String> PAYLOAD_CLASS = ConfigOptions
+      .key("write.payload.class")
+      .stringType()
+      .defaultValue(OverwriteWithLatestAvroPayload.class.getName())
+      .withDescription("Payload class used. Override this, if you like to roll your own merge logic, when upserting/inserting.\n"
+          + "This will render any value set for the option in-effective");
+
+  /**
+   * Flag to indicate whether to drop duplicates upon insert.
+   * By default insert will accept duplicates, to gain extra performance.
+   */
+  public static final ConfigOption<Boolean> INSERT_DROP_DUPS = ConfigOptions
+      .key("write.insert.drop.duplicates")
+      .booleanType()
+      .defaultValue(false)
+      .withDescription("Flag to indicate whether to drop duplicates upon insert.\n"
+          + "By default insert will accept duplicates, to gain extra performance");
+
+  public static final ConfigOption<Integer> RETRY_TIMES = ConfigOptions
+      .key("write.retry.times")
+      .intType()
+      .defaultValue(3)
+      .withDescription("Flag to indicate how many times streaming job should retry for a failed checkpoint batch.\n"
+          + "By default 3");
+
+  public static final ConfigOption<Long> RETRY_INTERVAL_MS = ConfigOptions
+      .key("write.retry.interval.ms")
+      .longType()
+      .defaultValue(2000L)
+      .withDescription("Flag to indicate how long (by millisecond) before a retry should issued for failed checkpoint batch.\n"
+          + "By default 2000 and it will be doubled by every retry");
+
+  public static final ConfigOption<Boolean> IGNORE_FAILED_BATCH = ConfigOptions
+      .key("write.ignore.failed.batch")

Review comment:
       Remove 'batch' because Flink may evolve to streaming write in the near future.
   




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



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

Posted by GitBox <gi...@apache.org>.
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.




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



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

Posted by GitBox <gi...@apache.org>.
garyli1019 commented on a change in pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#discussion_r560683106



##########
File path: hudi-common/src/main/java/org/apache/hudi/common/model/OverwriteWithLatestAvroPayload.java
##########
@@ -79,7 +79,12 @@ public OverwriteWithLatestAvroPayload preCombine(OverwriteWithLatestAvroPayload
    * @returns {@code true} if record represents a delete record. {@code false} otherwise.
    */
   protected boolean isDeleteRecord(GenericRecord genericRecord) {
-    Object deleteMarker = genericRecord.get("_hoodie_is_deleted");
+    final String isDeleteKey = "_hoodie_is_deleted";
+    // Modify to be compatible with old version Avro.
+    if (genericRecord.getSchema().getField(isDeleteKey) == null) {

Review comment:
       What does the old version Avro mean here? This seems like being handled by the check on return

##########
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:
       this is also available in `HoodieWriteConfig`, payload, kengen as well. If the config is commonly used by both engines, I think we could add new items to `HoodieWriteConfig` as well.

##########
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")
+      .stringType()
+      .defaultValue("ts")
+      .withDescription("Field used in preCombining before actual write. When two records have the same\n"
+          + "key value, we will pick the one with the largest value for the precombine field,\n"
+          + "determined by Object.compareTo(..)");
+
+  public static final ConfigOption<String> PAYLOAD_CLASS = ConfigOptions
+      .key("write.payload.class")
+      .stringType()
+      .defaultValue(OverwriteWithLatestAvroPayload.class.getName())
+      .withDescription("Payload class used. Override this, if you like to roll your own merge logic, when upserting/inserting.\n"
+          + "This will render any value set for the option in-effective");
+
+  /**
+   * Flag to indicate whether to drop duplicates upon insert.
+   * By default insert will accept duplicates, to gain extra performance.
+   */
+  public static final ConfigOption<Boolean> INSERT_DROP_DUPS = ConfigOptions
+      .key("write.insert.drop.duplicates")
+      .booleanType()
+      .defaultValue(false)
+      .withDescription("Flag to indicate whether to drop duplicates upon insert.\n"
+          + "By default insert will accept duplicates, to gain extra performance");
+
+  public static final ConfigOption<Integer> RETRY_TIMES = ConfigOptions
+      .key("write.retry.times")
+      .intType()
+      .defaultValue(3)
+      .withDescription("Flag to indicate how many times streaming job should retry for a failed checkpoint batch.\n"
+          + "By default 3");
+
+  public static final ConfigOption<Long> RETRY_INTERVAL_MS = ConfigOptions
+      .key("write.retry.interval.ms")
+      .longType()
+      .defaultValue(2000L)
+      .withDescription("Flag to indicate how long (by millisecond) before a retry should issued for failed checkpoint batch.\n"
+          + "By default 2000 and it will be doubled by every retry");
+
+  public static final ConfigOption<Boolean> IGNORE_FAILED_BATCH = ConfigOptions
+      .key("write.ignore.failed.batch")
+      .booleanType()
+      .defaultValue(true)
+      .withDescription("Flag to indicate whether to ignore any non exception error (e.g. writestatus error). within a checkpoint batch.\n"
+          + "By default true (in favor of streaming progressing over data integrity)");
+
+  public static final ConfigOption<String> RECORD_KEY_FIELD = ConfigOptions
+      .key(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY)
+      .stringType()
+      .defaultValue("uuid")
+      .withDescription("Record key field. Value to be used as the `recordKey` component of `HoodieKey`.\n"
+          + "Actual value will be obtained by invoking .toString() on the field value. Nested fields can be specified using "
+          + "the dot notation eg: `a.b.c`");
+
+  public static final ConfigOption<String> PARTITION_PATH_FIELD = ConfigOptions
+      .key(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY)
+      .stringType()
+      .defaultValue("partition-path")
+      .withDescription("Partition path field. Value to be used at the `partitionPath` component of `HoodieKey`.\n"
+          + "Actual value obtained by invoking .toString()");
+
+  public static final ConfigOption<String> KEYGEN_CLASS = ConfigOptions
+      .key("hoodie.datasource.write.keygenerator.class")
+      .stringType()
+      .defaultValue(SimpleAvroKeyGenerator.class.getName())
+      .withDescription("Key generator class, that implements will extract the key out of incoming record");
+
+  public static final ConfigOption<Integer> WRITE_TASK_PARALLELISM = ConfigOptions
+      .key("write.task.parallelism")
+      .intType()
+      .defaultValue(4)
+      .withDescription("Parallelism of tasks that do actual write, default is 4");
+

Review comment:
       maybe add another config to specify index type?

##########
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(
+              String.format("Failed to checkpoint Instant %s for source %s",
+                  this.inFlightInstant, this.getClass().getSimpleName()), throwable));
+    }
+  }
+
+  @Override
+  public void notifyCheckpointComplete(long checkpointId) {
+    // start to commit the instant.
+    checkAndCommitWithRetry();
+  }
+
+  @Override
+  public void notifyCheckpointAborted(long checkpointId) {
+    Preconditions.checkState(inFlightCheckpoint == checkpointId,
+        "The aborted checkpoint should always be the last checkpoint");
+    checkAndForceCommit("The last checkpoint was aborted, roll back the last write and throw");
+  }
+
+  @Override
+  public void resetToCheckpoint(long l, @Nullable byte[] checkpointData) throws Exception {
+    if (checkpointData != null) {
+      // restore when any checkpoint completed
+      deserializeCheckpointAndRestore(checkpointData);
+    }
+  }
+
+  @Override
+  public void handleEventFromOperator(int i, OperatorEvent operatorEvent) {
+    // no event to handle
+    Preconditions.checkState(operatorEvent instanceof BatchWriteSuccessEvent,
+        "The coordinator can only handle BatchWriteSuccessEvent");
+    BatchWriteSuccessEvent event = (BatchWriteSuccessEvent) operatorEvent;
+    Preconditions.checkState(event.getInstantTime().equals(this.inFlightInstant),
+        String.format("Receive an unexpected event for instant %s from task %d",
+            event.getInstantTime(), event.getTaskID()));
+    this.eventBuffer[event.getTaskID()] = event;
+  }
+
+  @Override
+  public void subtaskFailed(int i, @Nullable Throwable throwable) {
+    // no operation
+  }
+
+  @Override
+  public void subtaskReset(int i, long l) {
+    // no operation
+  }
+
+  // -------------------------------------------------------------------------
+  //  Utilities
+  // -------------------------------------------------------------------------
+
+  @SuppressWarnings("rawtypes")
+  private void initWriteClient() {
+    writeClient = new HoodieFlinkWriteClient(
+        new HoodieFlinkEngineContext(new FlinkTaskContextSupplier(null)),
+        StreamerUtil.getHoodieClientConfig(this.conf),
+        true);
+  }
+
+  private void initTable() throws IOException {
+    final String basePath = this.conf.getString(FlinkOptions.PATH);
+    final org.apache.hadoop.conf.Configuration hadoopConf = StreamerUtil.getHadoopConf();
+    // Hadoop FileSystem
+    try (FileSystem fs = FSUtils.getFs(basePath, hadoopConf)) {
+      if (!fs.exists(new Path(basePath, HoodieTableMetaClient.METAFOLDER_NAME))) {
+        HoodieTableMetaClient.initTableType(
+            hadoopConf,
+            basePath,
+            HoodieTableType.valueOf(this.conf.getString(FlinkOptions.TABLE_TYPE)),
+            this.conf.getString(FlinkOptions.TABLE_NAME),
+            "archived",
+            this.conf.getString(FlinkOptions.PAYLOAD_CLASS),
+            1);
+        LOG.info("Table initialized");
+      } else {
+        LOG.info("Table [{}/{}] already exists, no need to initialize the table",
+            basePath, this.conf.getString(FlinkOptions.TABLE_NAME));
+      }
+    }
+  }
+
+  static byte[] readBytes(DataInputStream in, int size) throws IOException {
+    byte[] bytes = new byte[size];
+    in.readFully(bytes);
+    return bytes;
+  }
+
+  /**
+   * Serialize the coordinator state. The current implementation may not be super efficient,
+   * but it should not matter that much because most of the state should be rather small.
+   * Large states themselves may already be a problem regardless of how the serialization
+   * is implemented.
+   *
+   * @return A byte array containing the serialized state of the source coordinator.
+   * @throws IOException When something goes wrong in serialization.
+   */
+  private byte[] writeCheckpointBytes() throws IOException {
+    try (ByteArrayOutputStream baos = new ByteArrayOutputStream();
+         DataOutputStream out = new DataOutputViewStreamWrapper(baos)) {
+
+      out.writeLong(this.inFlightCheckpoint);
+      byte[] serializedInstant = this.inFlightInstant.getBytes();
+      out.writeInt(serializedInstant.length);
+      out.write(serializedInstant);
+      out.flush();
+      return baos.toByteArray();
+    }
+  }
+
+  /**
+   * Restore the state of this source coordinator from the state bytes.
+   *
+   * @param bytes The checkpoint bytes that was returned from {@link #writeCheckpointBytes()}
+   * @throws Exception When the deserialization failed.
+   */
+  private void deserializeCheckpointAndRestore(byte[] bytes) throws Exception {
+    try (ByteArrayInputStream bais = new ByteArrayInputStream(bytes);
+         DataInputStream in = new DataInputViewStreamWrapper(bais)) {
+      long checkpointID = in.readLong();
+      int serializedInstantSize = in.readInt();
+      byte[] serializedInstant = readBytes(in, serializedInstantSize);
+      this.inFlightCheckpoint = checkpointID;
+      this.inFlightInstant = new String(serializedInstant);
+    }
+  }
+
+  private void reset() {
+    this.inFlightInstant = "";
+    this.eventBuffer = new BatchWriteSuccessEvent[this.parallelism];
+  }
+
+  private void checkAndForceCommit(String errMsg) {
+    if (!checkReady()) {
+      // forced but still has inflight instant
+      String inflightInstant = writeClient.getInflightAndRequestedInstant(this.conf.getString(FlinkOptions.TABLE_TYPE));
+      if (inflightInstant != null) {
+        assert inflightInstant.equals(this.inFlightInstant);
+        writeClient.rollback(this.inFlightInstant);
+        throw new HoodieException(errMsg);
+      }
+      if (Arrays.stream(eventBuffer).allMatch(Objects::isNull)) {
+        // The last checkpoint finished successfully.
+        return;
+      }
+    }
+    doCommit();
+  }
+
+  private void checkAndCommitWithRetry() {
+    int retryTimes = this.conf.getInteger(FlinkOptions.RETRY_TIMES);
+    long retryIntervalMillis = this.conf.getLong(FlinkOptions.RETRY_INTERVAL_MS);
+    int tryTimes = 0;
+    while (tryTimes++ < retryTimes) {
+      try {
+        if (!checkReady()) {
+          // Do not throw if the try times expires but the event buffer are still not ready,
+          // because we have a force check when next checkpoint starts.
+          waitFor(retryIntervalMillis);
+          continue;
+        }
+        doCommit();
+        return;
+      } catch (Throwable throwable) {
+        String cause = throwable.getCause() == null ? "" : throwable.getCause().toString();
+        LOG.warn("Try to commit the instant {} failed, with times {} and cause {}", this.inFlightInstant, tryTimes, cause);
+        if (tryTimes == retryTimes) {
+          throw new HoodieException(throwable);
+        }
+        waitFor(retryIntervalMillis);
+      }
+    }
+  }
+
+  private void waitFor(long intervalMillis) {
+    try {
+      TimeUnit.MILLISECONDS.sleep(intervalMillis);
+    } catch (InterruptedException e) {
+      LOG.error("Thread interrupted while waiting to retry the instant commits");
+      throw new HoodieException(e);
+    }
+  }
+
+  /** Checks the buffer is ready to commit. */
+  private boolean checkReady() {
+    return Arrays.stream(eventBuffer).allMatch(event ->
+        event != null && event.getInstantTime().equals(this.inFlightInstant));
+  }
+
+  /** Performs the actual commit action. */
+  private void doCommit() {
+    List<WriteStatus> writeResults = Arrays.stream(eventBuffer)
+        .map(BatchWriteSuccessEvent::getWriteStatuses)
+        .flatMap(Collection::stream)
+        .collect(Collectors.toList());
+
+    if (writeResults.size() == 0) {
+      // No data has written, clear the metadata file
+      this.writeClient.deletePendingInstant(this.conf.getString(FlinkOptions.TABLE_TYPE), this.inFlightInstant);
+      reset();
+      return;
+    }
+
+    // commit or rollback
+    long totalErrorRecords = writeResults.stream().map(WriteStatus::getTotalErrorRecords).reduce(Long::sum).orElse(0L);
+    long totalRecords = writeResults.stream().map(WriteStatus::getTotalRecords).reduce(Long::sum).orElse(0L);
+    boolean hasErrors = totalErrorRecords > 0;
+
+    if (!hasErrors || this.conf.getBoolean(FlinkOptions.IGNORE_FAILED_BATCH)) {
+      HashMap<String, String> checkpointCommitMetadata = new HashMap<>();
+      if (hasErrors) {
+        LOG.warn("Some records failed to merge but forcing commit since commitOnErrors set to true. Errors/Total="
+            + totalErrorRecords + "/" + totalRecords);
+      }
+
+      boolean success = writeClient.commit(this.inFlightInstant, writeResults, Option.of(checkpointCommitMetadata));
+      if (success) {
+        reset();
+        LOG.info("Commit instant [{}] success!", this.inFlightInstant);
+      } else {
+        throw new HoodieException(String.format("Commit instant [%s] failed!", this.inFlightInstant));
+      }
+    } else {
+      LOG.error("Error when writing. Errors/Total=" + totalErrorRecords + "/" + totalRecords);
+      LOG.error("The first 100 error messages");
+      writeResults.stream().filter(WriteStatus::hasErrors).limit(100).forEach(ws -> {
+        LOG.error("Global error for partition path {} and fileID {}: {}",
+            ws.getGlobalError(), ws.getPartitionPath(), ws.getFileId());
+        if (ws.getErrors().size() > 0) {
+          ws.getErrors().forEach((key, value) -> LOG.trace("Error for key:" + key + " and value " + value));
+        }
+      });
+      // Rolls back instant
+      writeClient.rollback(this.inFlightInstant);
+      throw new HoodieException(String.format("Commit instant [%s] failed and rolled back !", this.inFlightInstant));
+    }
+  }
+
+  @VisibleForTesting
+  public BatchWriteSuccessEvent[] getEventBuffer() {
+    return eventBuffer;
+  }
+
+  @VisibleForTesting
+  public String getInFlightInstant() {

Review comment:
       is this only visible for testing? IIUC, the actual writer need to know the current instant and add `_hoodie_commit_time` field to the record. How did the writer know the current instant?

##########
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:
       Can we add some docs here. Still a little bit confused about how this will be used.




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



[GitHub] [hudi] codecov-io edited a comment on pull request #2430: [HUDI-1522] Add a new pipeline for Flink writer

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#issuecomment-757736411






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



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

Posted by GitBox <gi...@apache.org>.
danny0405 commented on a change in pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#discussion_r557013066



##########
File path: hudi-flink/src/main/java/org/apache/hudi/streamer/OperationConverter.java
##########
@@ -0,0 +1,32 @@
+/*
+ * 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. */

Review comment:
       We can, if we fond more usages of this class, in current code base, only 2 places use it, and the class itself is pretty simple, it does not deserve to move the class to common utils.




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



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

Posted by GitBox <gi...@apache.org>.
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:
users@infra.apache.org



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

Posted by GitBox <gi...@apache.org>.
danny0405 commented on a change in pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#discussion_r560728220



##########
File path: hudi-common/src/main/java/org/apache/hudi/common/model/OverwriteWithLatestAvroPayload.java
##########
@@ -79,7 +79,12 @@ public OverwriteWithLatestAvroPayload preCombine(OverwriteWithLatestAvroPayload
    * @returns {@code true} if record represents a delete record. {@code false} otherwise.
    */
   protected boolean isDeleteRecord(GenericRecord genericRecord) {
-    Object deleteMarker = genericRecord.get("_hoodie_is_deleted");
+    final String isDeleteKey = "_hoodie_is_deleted";
+    // Modify to be compatible with old version Avro.
+    if (genericRecord.getSchema().getField(isDeleteKey) == null) {

Review comment:
       Hmm, i mean the new version Avro, let me fix the comment.




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



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

Posted by GitBox <gi...@apache.org>.
yanghua commented on a change in pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#discussion_r557331716



##########
File path: hudi-flink/pom.xml
##########
@@ -124,28 +124,77 @@
       <artifactId>kafka-clients</artifactId>
       <version>${kafka.version}</version>
     </dependency>
+    <dependency>
+      <groupId>org.apache.flink</groupId>
+      <artifactId>flink-hadoop-compatibility_${scala.binary.version}</artifactId>
+      <version>${flink.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.flink</groupId>
+      <artifactId>flink-avro</artifactId>
+      <version>${flink.version}</version>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.flink</groupId>
+      <artifactId>flink-json</artifactId>
+      <version>${flink.version}</version>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.flink</groupId>
+      <artifactId>flink-table-common</artifactId>
+      <version>${flink.version}</version>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.flink</groupId>
+      <artifactId>flink-table-runtime-blink_${scala.binary.version}</artifactId>

Review comment:
       I mean if users only change the Flink version placeholder(`${flink.version}`), it will still cause the compile error, right? Maybe we need to use `<profile>` for the bundle module or another solution?




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



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

Posted by GitBox <gi...@apache.org>.
danny0405 commented on a change in pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#discussion_r556381921



##########
File path: hudi-flink/src/main/java/org/apache/hudi/streamer/OperationConverter.java
##########
@@ -0,0 +1,32 @@
+/*
+ * 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. */

Review comment:
       `org.apache.hudi.utilities.deltastreamer` `OperationConverter` is in a different module, i would suggest to not add dependency to that. The `hoodie-flink` module should not depend on the `hudi-utilities`.




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



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

Posted by GitBox <gi...@apache.org>.
danny0405 commented on a change in pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#discussion_r559435103



##########
File path: hudi-flink/src/main/java/org/apache/hudi/streamer/Config.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 Config extends Configuration {

Review comment:
       Fine with the suggest name.




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



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

Posted by GitBox <gi...@apache.org>.
danny0405 commented on a change in pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#discussion_r560747457



##########
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(FlinkOptions.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(FlinkOptions.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(toHoodieRecord(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(FlinkOptions.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 toHoodieRecord(I record) throws IOException {
+    boolean shouldCombine = this.config.getBoolean(FlinkOptions.INSERT_DROP_DUPS)
+        || WriteOperationType.fromValue(this.config.getString(FlinkOptions.OPERATION)) == WriteOperationType.UPSERT;
+    GenericRecord gr = (GenericRecord) this.converter.convert(this.avroSchema, record);
+    HoodieRecordPayload payload = shouldCombine
+        ? StreamerUtil.createPayload(this.config.getString(FlinkOptions.PAYLOAD_CLASS), gr,
+        (Comparable) HoodieAvroUtils.getNestedFieldVal(gr, this.config.getString(FlinkOptions.PRECOMBINE_FIELD), false))
+        : StreamerUtil.createPayload(this.config.getString(FlinkOptions.PAYLOAD_CLASS), gr);
+    return new HoodieRecord<>(keyGenerator.getKey(gr), payload);
+  }
+
+  private void flushBuffer() {
+    final List<WriteStatus> writeStatus;
+    if (buffer.size() > 0) {
+      writeStatus = writeFunction.apply(buffer, currentInstant);
+      buffer.clear();
+    } else {
+      LOG.info("No data to write in subtask [{}] for instant [{}]", taskID, currentInstant);
+      writeStatus = Collections.emptyList();
+    }
+    this.eventGateway.sendEventToCoordinator(new BatchWriteSuccessEvent(this.taskID, currentInstant, writeStatus));

Review comment:
       Failover when next checkpoint starts. There is a check in the `StreamWriteOperatorCoordinator.checkpointCoordinator`.




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



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

Posted by GitBox <gi...@apache.org>.
danny0405 commented on a change in pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#discussion_r559431430



##########
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:
       Agree, thanks ~




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



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

Posted by GitBox <gi...@apache.org>.
danny0405 commented on a change in pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#discussion_r559432105



##########
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:
       Yes, removed.




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



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

Posted by GitBox <gi...@apache.org>.
yanghua commented on a change in pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#discussion_r560732981



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

Review comment:
       IMO, one line can do this definition. It's not too long, but introduced a new code style.




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



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

Posted by GitBox <gi...@apache.org>.
yanghua commented on a change in pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#discussion_r561835785



##########
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:
       Where do we use this class?




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



[GitHub] [hudi] codecov-io edited a comment on pull request #2430: [HUDI-1522] Add a new pipeline for Flink writer

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#issuecomment-757736411






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



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

Posted by GitBox <gi...@apache.org>.
yanghua commented on a change in pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#discussion_r561841835



##########
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:
       Yes, I know this, but still brings external dependency for the DataStream users.  There are two concerns:
   
   1) This brings an additional burden for sharing the same abstraction with the current implementation;
   2) RowType makes the number of compatible Flink versions less.




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



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

Posted by GitBox <gi...@apache.org>.
yanghua commented on a change in pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#discussion_r561716925



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

Review comment:
       So can we make `COPY_ON_WRITE` and `copy_on_write` equivalence?

##########
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:
       > The config options in HoodieWriteConfig are too verbose. 
   
   That's true.
   
   But for the same configuration, if we have multiple pointers to it, it will bring the risk of losing control. In the future, if we have some changes to this configuration, such as making it depreciated. We may lose our unified control over it.

##########
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'd better find a way to unify the config names, e.g. do some refactor?

##########
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")
+      .stringType()
+      .defaultValue("ts")
+      .withDescription("Field used in preCombining before actual write. When two records have the same\n"
+          + "key value, we will pick the one with the largest value for the precombine field,\n"
+          + "determined by Object.compareTo(..)");
+
+  public static final ConfigOption<String> PAYLOAD_CLASS = ConfigOptions
+      .key("write.payload.class")
+      .stringType()
+      .defaultValue(OverwriteWithLatestAvroPayload.class.getName())
+      .withDescription("Payload class used. Override this, if you like to roll your own merge logic, when upserting/inserting.\n"
+          + "This will render any value set for the option in-effective");
+
+  /**
+   * Flag to indicate whether to drop duplicates upon insert.
+   * By default insert will accept duplicates, to gain extra performance.
+   */
+  public static final ConfigOption<Boolean> INSERT_DROP_DUPS = ConfigOptions
+      .key("write.insert.drop.duplicates")
+      .booleanType()
+      .defaultValue(false)
+      .withDescription("Flag to indicate whether to drop duplicates upon insert.\n"
+          + "By default insert will accept duplicates, to gain extra performance");
+
+  public static final ConfigOption<Integer> RETRY_TIMES = ConfigOptions
+      .key("write.retry.times")
+      .intType()
+      .defaultValue(3)
+      .withDescription("Flag to indicate how many times streaming job should retry for a failed checkpoint batch.\n"
+          + "By default 3");
+
+  public static final ConfigOption<Long> RETRY_INTERVAL_MS = ConfigOptions
+      .key("write.retry.interval.ms")
+      .longType()
+      .defaultValue(2000L)
+      .withDescription("Flag to indicate how long (by millisecond) before a retry should issued for failed checkpoint batch.\n"
+          + "By default 2000 and it will be doubled by every retry");
+
+  public static final ConfigOption<Boolean> IGNORE_FAILED_BATCH = ConfigOptions
+      .key("write.ignore.failed.batch")
+      .booleanType()
+      .defaultValue(true)
+      .withDescription("Flag to indicate whether to ignore any non exception error (e.g. writestatus error). within a checkpoint batch.\n"
+          + "By default true (in favor of streaming progressing over data integrity)");
+
+  public static final ConfigOption<String> RECORD_KEY_FIELD = ConfigOptions
+      .key(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY)
+      .stringType()
+      .defaultValue("uuid")
+      .withDescription("Record key field. Value to be used as the `recordKey` component of `HoodieKey`.\n"
+          + "Actual value will be obtained by invoking .toString() on the field value. Nested fields can be specified using "
+          + "the dot notation eg: `a.b.c`");
+
+  public static final ConfigOption<String> PARTITION_PATH_FIELD = ConfigOptions
+      .key(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY)
+      .stringType()
+      .defaultValue("partition-path")
+      .withDescription("Partition path field. Value to be used at the `partitionPath` component of `HoodieKey`.\n"
+          + "Actual value obtained by invoking .toString()");
+
+  public static final ConfigOption<String> KEYGEN_CLASS = ConfigOptions
+      .key("hoodie.datasource.write.keygenerator.class")

Review comment:
       any response?

##########
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:
       Actually, do not know we really need to use `RowType`, only for convenience to integrate with `table/sql`?

##########
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:
       Where do we use this class?

##########
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:
       Yes, I know this, but still brings external dependency for the DataStream users.  There are two concerns:
   
   1) This brings an additional burden for sharing the same abstraction with the current implementation;
   2) RowType makes the number of compatible Flink versions less.

##########
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:
       `block` seems hard to understand. Any better word?

##########
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:
       The reason? why disable the operator chain? 

##########
File path: hudi-flink/src/main/java/org/apache/hudi/streamer/HoodieFlinkStreamer.java
##########
@@ -0,0 +1,108 @@
+/*
+ * 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.client.WriteStatus;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.operator.InstantGenerateOperator;
+import org.apache.hudi.operator.KeyedWriteProcessFunction;
+import org.apache.hudi.operator.KeyedWriteProcessOperator;
+import org.apache.hudi.sink.CommitSink;
+import org.apache.hudi.source.JsonStringToHoodieRecordMapFunction;
+import org.apache.hudi.util.StreamerUtil;
+
+import com.beust.jcommander.JCommander;
+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.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 java.util.List;
+import java.util.Objects;
+
+/**
+ * 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 HoodieFlinkStreamer {
+  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:
       ditto

##########
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:
       unnecessary comment for the constructor? it's easy to understand.

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

Review comment:
       typo: `even` -> `event`

##########
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:
       Are you really so young?

##########
File path: hudi-flink/src/test/resources/test_read_schema.avsc
##########
@@ -0,0 +1,45 @@
+/*
+ * 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.
+ */
+{
+  "type" : "record",
+  "name" : "record",
+  "fields" : [ {

Review comment:
       Can we follow the same style about the test schema, align with the existed file, e.g. `HoodieCleanerPlan.avsc`




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



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

Posted by GitBox <gi...@apache.org>.
danny0405 commented on a change in pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#discussion_r557012704



##########
File path: hudi-flink/src/main/java/org/apache/hudi/util/StreamerUtil.java
##########
@@ -81,16 +103,50 @@ public static DFSPropertiesConfiguration readConfig(FileSystem fs, Path cfgPath,
     return conf;
   }
 
-  public static Configuration getHadoopConf() {
-    return new Configuration();
+  public static org.apache.hadoop.conf.Configuration getHadoopConf() {
+    // create HiveConf from hadoop configuration with hadoop conf directory configured.
+    org.apache.hadoop.conf.Configuration hadoopConf = null;
+    for (String possibleHadoopConfPath : HadoopUtils.possibleHadoopConfPaths(new Configuration())) {

Review comment:
       The method firstly find the specified path `fs.hdfs.hadoopconf`, then directory `HADOOP_CONF_DIR` `HADOOP_HOME/conf` `HADOOP_HOME/etc/hadoop` from the system environment.
   
   Even if storage is separated from computing, the `FileSystem` we created is still correct, if we split the hadoop conf files correctly.




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



[GitHub] [hudi] codecov-io edited a comment on pull request #2430: [HUDI-1522] Remove the single parallelism operator from the Flink writer

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#issuecomment-757736411


   # [Codecov](https://codecov.io/gh/apache/hudi/pull/2430?src=pr&el=h1) Report
   > Merging [#2430](https://codecov.io/gh/apache/hudi/pull/2430?src=pr&el=desc) (7961488) into [master](https://codecov.io/gh/apache/hudi/commit/7ce3ac778eb475bf23ffa31243dc0843ec7d089a?el=desc) (7ce3ac7) will **decrease** coverage by `6.54%`.
   > The diff coverage is `59.79%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/hudi/pull/2430/graphs/tree.svg?width=650&height=150&src=pr&token=VTTXabwbs2)](https://codecov.io/gh/apache/hudi/pull/2430?src=pr&el=tree)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #2430      +/-   ##
   ============================================
   - Coverage     50.76%   44.22%   -6.55%     
   + Complexity     3063     2265     -798     
   ============================================
     Files           419      329      -90     
     Lines         18777    14395    -4382     
     Branches       1918     1379     -539     
   ============================================
   - Hits           9533     6366    -3167     
   + Misses         8468     7559     -909     
   + Partials        776      470     -306     
   ```
   
   | Flag | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | hudicli | `37.26% <ø> (ø)` | `0.00 <ø> (ø)` | |
   | hudiclient | `100.00% <ø> (ø)` | `0.00 <ø> (ø)` | |
   | hudicommon | `52.05% <33.33%> (-0.03%)` | `0.00 <2.00> (ø)` | |
   | hudiflink | `53.98% <59.95%> (+43.78%)` | `0.00 <56.00> (ø)` | |
   | hudihadoopmr | `?` | `?` | |
   | hudisparkdatasource | `?` | `?` | |
   | hudisync | `?` | `?` | |
   | huditimelineservice | `?` | `?` | |
   | hudiutilities | `9.68% <ø> (-59.75%)` | `0.00 <ø> (ø)` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/hudi/pull/2430?src=pr&el=tree) | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | [...ain/java/org/apache/hudi/avro/HoodieAvroUtils.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvYXZyby9Ib29kaWVBdnJvVXRpbHMuamF2YQ==) | `57.07% <ø> (-0.21%)` | `41.00 <0.00> (ø)` | |
   | [...main/java/org/apache/hudi/HoodieFlinkStreamer.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS1mbGluay9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvaHVkaS9Ib29kaWVGbGlua1N0cmVhbWVyLmphdmE=) | `0.00% <0.00%> (ø)` | `0.00 <0.00> (ø)` | |
   | [.../org/apache/hudi/operator/StreamWriteOperator.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS1mbGluay9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvaHVkaS9vcGVyYXRvci9TdHJlYW1Xcml0ZU9wZXJhdG9yLmphdmE=) | `0.00% <0.00%> (ø)` | `0.00 <0.00> (?)` | |
   | [...ache/hudi/operator/StreamWriteOperatorFactory.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS1mbGluay9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvaHVkaS9vcGVyYXRvci9TdHJlYW1Xcml0ZU9wZXJhdG9yRmFjdG9yeS5qYXZh) | `0.00% <0.00%> (ø)` | `0.00 <0.00> (?)` | |
   | [...i/common/model/OverwriteWithLatestAvroPayload.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL21vZGVsL092ZXJ3cml0ZVdpdGhMYXRlc3RBdnJvUGF5bG9hZC5qYXZh) | `60.00% <33.33%> (-4.71%)` | `10.00 <2.00> (ø)` | |
   | [...c/main/java/org/apache/hudi/util/StreamerUtil.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS1mbGluay9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvaHVkaS91dGlsL1N0cmVhbWVyVXRpbC5qYXZh) | `35.89% <47.36%> (+24.26%)` | `9.00 <8.00> (+6.00)` | |
   | [.../hudi/operator/StreamWriteOperatorCoordinator.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS1mbGluay9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvaHVkaS9vcGVyYXRvci9TdHJlYW1Xcml0ZU9wZXJhdG9yQ29vcmRpbmF0b3IuamF2YQ==) | `63.52% <63.52%> (ø)` | `25.00 <25.00> (?)` | |
   | [.../org/apache/hudi/operator/StreamWriteFunction.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS1mbGluay9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvaHVkaS9vcGVyYXRvci9TdHJlYW1Xcml0ZUZ1bmN0aW9uLmphdmE=) | `68.67% <68.67%> (ø)` | `14.00 <14.00> (?)` | |
   | [...n/java/org/apache/hudi/operator/HoodieOptions.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS1mbGluay9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvaHVkaS9vcGVyYXRvci9Ib29kaWVPcHRpb25zLmphdmE=) | `74.35% <74.35%> (ø)` | `3.00 <3.00> (?)` | |
   | [...he/hudi/operator/event/BatchWriteSuccessEvent.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS1mbGluay9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvaHVkaS9vcGVyYXRvci9ldmVudC9CYXRjaFdyaXRlU3VjY2Vzc0V2ZW50LmphdmE=) | `100.00% <100.00%> (ø)` | `4.00 <4.00> (?)` | |
   | ... and [140 more](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree-more) | |
   


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



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

Posted by GitBox <gi...@apache.org>.
danny0405 commented on a change in pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#discussion_r558065700



##########
File path: hudi-common/src/main/java/org/apache/hudi/common/model/OverwriteWithLatestAvroPayload.java
##########
@@ -79,6 +79,11 @@ public OverwriteWithLatestAvroPayload preCombine(OverwriteWithLatestAvroPayload
    * @returns {@code true} if record represents a delete record. {@code false} otherwise.
    */
   protected boolean isDeleteRecord(GenericRecord genericRecord) {
+    final String isDeleteKey = "_hoodie_is_deleted";
+    // Modify to be compatible with old version Avro.
+    if (genericRecord.getSchema().getField(isDeleteKey) == null) {
+      return false;
+    }
     Object deleteMarker = genericRecord.get("_hoodie_is_deleted");

Review comment:
       Okey, got your idea, thanks ~




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



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

Posted by GitBox <gi...@apache.org>.
yanghua commented on a change in pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#discussion_r557337056



##########
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:
       fine, it's not a key issue, different thought is OK.




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



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

Posted by GitBox <gi...@apache.org>.
yanghua commented on a change in pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#discussion_r557306549



##########
File path: hudi-flink/src/main/java/org/apache/hudi/util/StreamerUtil.java
##########
@@ -81,16 +103,50 @@ public static DFSPropertiesConfiguration readConfig(FileSystem fs, Path cfgPath,
     return conf;
   }
 
-  public static Configuration getHadoopConf() {
-    return new Configuration();
+  public static org.apache.hadoop.conf.Configuration getHadoopConf() {
+    // create HiveConf from hadoop configuration with hadoop conf directory configured.
+    org.apache.hadoop.conf.Configuration hadoopConf = null;
+    for (String possibleHadoopConfPath : HadoopUtils.possibleHadoopConfPaths(new Configuration())) {

Review comment:
       > The method firstly find the specified path `fs.hdfs.hadoopconf`, then directory `HADOOP_CONF_DIR` `HADOOP_HOME/conf` `HADOOP_HOME/etc/hadoop` from the system environment.
   > 
   
   I have watched the source code before raising this concern.
   
   > Even if storage is separated from computing, the `FileSystem` we created is still correct, if we split the hadoop conf files correctly.
   > 
   > In any case, we should not pass an empty hadoop configuration.
   
   I mean, do we allow the user's explicit parameter assignment as the highest priority? Greater than the default convention that some users may not know?




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



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

Posted by GitBox <gi...@apache.org>.
yanghua commented on a change in pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#discussion_r564523949



##########
File path: hudi-flink/src/main/java/org/apache/hudi/operator/FlinkOptions.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.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");
+
+  // ------------------------------------------------------------------------
+  //  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");

Review comment:
       remove `.` after `write`

##########
File path: hudi-flink/src/main/java/org/apache/hudi/operator/FlinkOptions.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.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");
+
+  // ------------------------------------------------------------------------
+  //  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")

Review comment:
       Is it also case sensitive?

##########
File path: hudi-flink/src/main/java/org/apache/hudi/operator/FlinkOptions.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.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");
+
+  // ------------------------------------------------------------------------
+  //  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")

Review comment:
       let us reduce the string literal via `HoodieTableType.COPY_ON_WRITE.name()`?

##########
File path: hudi-flink/src/main/java/org/apache/hudi/operator/StreamWriteOperatorCoordinator.java
##########
@@ -0,0 +1,413 @@
+/*
+ * 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(
+              String.format("Failed to checkpoint Instant %s for source %s",
+                  this.inFlightInstant, this.getClass().getSimpleName()), throwable));
+    }
+  }
+
+  @Override
+  public void checkpointComplete(long checkpointId) {
+    // start to commit the instant.
+    checkAndCommitWithRetry();
+  }
+
+  public void notifyCheckpointAborted(long checkpointId) {
+    Preconditions.checkState(inFlightCheckpoint == checkpointId,
+        "The aborted checkpoint should always be the last checkpoint");
+    checkAndForceCommit("The last checkpoint was aborted, roll back the last write and throw");
+  }
+
+  @Override
+  public void resetToCheckpoint(@Nullable byte[] checkpointData) throws Exception {
+    if (checkpointData != null) {
+      // restore when any checkpoint completed
+      deserializeCheckpointAndRestore(checkpointData);
+    }
+  }
+
+  @Override
+  public void handleEventFromOperator(int i, OperatorEvent operatorEvent) {
+    // no event to handle
+    Preconditions.checkState(operatorEvent instanceof BatchWriteSuccessEvent,
+        "The coordinator can only handle BatchWriteSuccessEvent");
+    BatchWriteSuccessEvent event = (BatchWriteSuccessEvent) operatorEvent;
+    Preconditions.checkState(event.getInstantTime().equals(this.inFlightInstant),
+        String.format("Receive an unexpected event for instant %s from task %d",
+            event.getInstantTime(), event.getTaskID()));
+    this.eventBuffer[event.getTaskID()] = event;
+  }
+
+  @Override
+  public void subtaskFailed(int i, @Nullable Throwable throwable) {
+    // no operation
+  }
+
+  // -------------------------------------------------------------------------
+  //  Utilities
+  // -------------------------------------------------------------------------
+
+  @SuppressWarnings("rawtypes")
+  private void initWriteClient() {
+    writeClient = new HoodieFlinkWriteClient(
+        new HoodieFlinkEngineContext(new FlinkTaskContextSupplier(null)),
+        StreamerUtil.getHoodieClientConfig(this.conf),
+        true);
+  }
+
+  private void initTable() throws IOException {
+    final String basePath = this.conf.getString(FlinkOptions.PATH);
+    final org.apache.hadoop.conf.Configuration hadoopConf = StreamerUtil.getHadoopConf();
+    // Hadoop FileSystem
+    try (FileSystem fs = FSUtils.getFs(basePath, hadoopConf)) {
+      if (!fs.exists(new Path(basePath, HoodieTableMetaClient.METAFOLDER_NAME))) {
+        HoodieTableMetaClient.initTableType(
+            hadoopConf,
+            basePath,
+            HoodieTableType.valueOf(this.conf.getString(FlinkOptions.TABLE_TYPE)),
+            this.conf.getString(FlinkOptions.TABLE_NAME),
+            "archived",
+            this.conf.getString(FlinkOptions.PAYLOAD_CLASS),
+            1);
+        LOG.info("Table initialized");
+      } else {
+        LOG.info("Table [{}/{}] already exists, no need to initialize the table",
+            basePath, this.conf.getString(FlinkOptions.TABLE_NAME));
+      }
+    }
+  }
+
+  static byte[] readBytes(DataInputStream in, int size) throws IOException {
+    byte[] bytes = new byte[size];
+    in.readFully(bytes);
+    return bytes;
+  }
+
+  /**
+   * Serialize the coordinator state. The current implementation may not be super efficient,
+   * but it should not matter that much because most of the state should be rather small.
+   * Large states themselves may already be a problem regardless of how the serialization
+   * is implemented.
+   *
+   * @return A byte array containing the serialized state of the source coordinator.
+   * @throws IOException When something goes wrong in serialization.
+   */
+  private byte[] writeCheckpointBytes() throws IOException {
+    try (ByteArrayOutputStream baos = new ByteArrayOutputStream();
+         DataOutputStream out = new DataOutputViewStreamWrapper(baos)) {
+
+      out.writeLong(this.inFlightCheckpoint);
+      byte[] serializedInstant = this.inFlightInstant.getBytes();
+      out.writeInt(serializedInstant.length);
+      out.write(serializedInstant);
+      out.flush();
+      return baos.toByteArray();
+    }
+  }
+
+  /**
+   * Restore the state of this source coordinator from the state bytes.
+   *
+   * @param bytes The checkpoint bytes that was returned from {@link #writeCheckpointBytes()}
+   * @throws Exception When the deserialization failed.
+   */
+  private void deserializeCheckpointAndRestore(byte[] bytes) throws Exception {
+    try (ByteArrayInputStream bais = new ByteArrayInputStream(bytes);
+         DataInputStream in = new DataInputViewStreamWrapper(bais)) {
+      long checkpointID = in.readLong();
+      int serializedInstantSize = in.readInt();
+      byte[] serializedInstant = readBytes(in, serializedInstantSize);
+      this.inFlightCheckpoint = checkpointID;
+      this.inFlightInstant = new String(serializedInstant);
+    }
+  }
+
+  private void reset() {
+    this.inFlightInstant = "";
+    this.eventBuffer = new BatchWriteSuccessEvent[this.parallelism];
+  }
+
+  private void checkAndForceCommit(String errMsg) {
+    if (!checkReady()) {
+      // forced but still has inflight instant
+      String inflightInstant = writeClient.getInflightAndRequestedInstant(this.conf.getString(FlinkOptions.TABLE_TYPE));
+      if (inflightInstant != null) {
+        assert inflightInstant.equals(this.inFlightInstant);
+        writeClient.rollback(this.inFlightInstant);
+        throw new HoodieException(errMsg);
+      }
+      if (Arrays.stream(eventBuffer).allMatch(Objects::isNull)) {
+        // The last checkpoint finished successfully.
+        return;
+      }
+    }
+    doCommit();
+  }
+
+  private void checkAndCommitWithRetry() {
+    int retryTimes = this.conf.getInteger(FlinkOptions.RETRY_TIMES);
+    long retryIntervalMillis = this.conf.getLong(FlinkOptions.RETRY_INTERVAL_MS);
+    int tryTimes = 0;
+    while (tryTimes++ < retryTimes) {
+      try {
+        if (!checkReady()) {
+          // Do not throw if the try times expires but the event buffer are still not ready,
+          // because we have a force check when next checkpoint starts.
+          waitFor(retryIntervalMillis);
+          continue;
+        }
+        doCommit();
+        return;
+      } catch (Throwable throwable) {
+        String cause = throwable.getCause() == null ? "" : throwable.getCause().toString();

Review comment:
       Did you check that how many exception categories? If there is not necessary to still retry, may we accept `fast failed` strategy?

##########
File path: hudi-flink/src/main/java/org/apache/hudi/operator/FlinkOptions.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.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");
+
+  // ------------------------------------------------------------------------
+  //  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")
+      .stringType()
+      .defaultValue("ts")
+      .withDescription("Field used in preCombining before actual write. When two records have the same\n"
+          + "key value, we will pick the one with the largest value for the precombine field,\n"
+          + "determined by Object.compareTo(..)");
+
+  public static final ConfigOption<String> PAYLOAD_CLASS = ConfigOptions
+      .key("write.payload.class")
+      .stringType()
+      .defaultValue(OverwriteWithLatestAvroPayload.class.getName())
+      .withDescription("Payload class used. Override this, if you like to roll your own merge logic, when upserting/inserting.\n"
+          + "This will render any value set for the option in-effective");
+
+  /**
+   * Flag to indicate whether to drop duplicates upon insert.
+   * By default insert will accept duplicates, to gain extra performance.
+   */
+  public static final ConfigOption<Boolean> INSERT_DROP_DUPS = ConfigOptions
+      .key("write.insert.drop.duplicates")
+      .booleanType()
+      .defaultValue(false)
+      .withDescription("Flag to indicate whether to drop duplicates upon insert.\n"
+          + "By default insert will accept duplicates, to gain extra performance");
+
+  public static final ConfigOption<Integer> RETRY_TIMES = ConfigOptions
+      .key("write.retry.times")
+      .intType()
+      .defaultValue(3)
+      .withDescription("Flag to indicate how many times streaming job should retry for a failed checkpoint batch.\n"
+          + "By default 3");
+
+  public static final ConfigOption<Long> RETRY_INTERVAL_MS = ConfigOptions
+      .key("write.retry.interval.ms")
+      .longType()
+      .defaultValue(2000L)
+      .withDescription("Flag to indicate how long (by millisecond) before a retry should issued for failed checkpoint batch.\n"
+          + "By default 2000 and it will be doubled by every retry");
+
+  public static final ConfigOption<Boolean> IGNORE_FAILED_BATCH = ConfigOptions
+      .key("write.ignore.failed.batch")

Review comment:
       `write.batch.ignore.failed` sounds better? This config option describes a boolean flag, right?

##########
File path: hudi-flink/src/main/java/org/apache/hudi/operator/StreamWriteOperatorCoordinator.java
##########
@@ -0,0 +1,413 @@
+/*
+ * 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(
+              String.format("Failed to checkpoint Instant %s for source %s",
+                  this.inFlightInstant, this.getClass().getSimpleName()), throwable));
+    }
+  }
+
+  @Override
+  public void checkpointComplete(long checkpointId) {
+    // start to commit the instant.
+    checkAndCommitWithRetry();
+  }
+
+  public void notifyCheckpointAborted(long checkpointId) {
+    Preconditions.checkState(inFlightCheckpoint == checkpointId,
+        "The aborted checkpoint should always be the last checkpoint");
+    checkAndForceCommit("The last checkpoint was aborted, roll back the last write and throw");
+  }
+
+  @Override
+  public void resetToCheckpoint(@Nullable byte[] checkpointData) throws Exception {
+    if (checkpointData != null) {
+      // restore when any checkpoint completed
+      deserializeCheckpointAndRestore(checkpointData);
+    }
+  }
+
+  @Override
+  public void handleEventFromOperator(int i, OperatorEvent operatorEvent) {
+    // no event to handle
+    Preconditions.checkState(operatorEvent instanceof BatchWriteSuccessEvent,
+        "The coordinator can only handle BatchWriteSuccessEvent");
+    BatchWriteSuccessEvent event = (BatchWriteSuccessEvent) operatorEvent;
+    Preconditions.checkState(event.getInstantTime().equals(this.inFlightInstant),
+        String.format("Receive an unexpected event for instant %s from task %d",
+            event.getInstantTime(), event.getTaskID()));
+    this.eventBuffer[event.getTaskID()] = event;
+  }
+
+  @Override
+  public void subtaskFailed(int i, @Nullable Throwable throwable) {
+    // no operation
+  }
+
+  // -------------------------------------------------------------------------
+  //  Utilities
+  // -------------------------------------------------------------------------
+
+  @SuppressWarnings("rawtypes")
+  private void initWriteClient() {
+    writeClient = new HoodieFlinkWriteClient(
+        new HoodieFlinkEngineContext(new FlinkTaskContextSupplier(null)),
+        StreamerUtil.getHoodieClientConfig(this.conf),
+        true);
+  }
+
+  private void initTable() throws IOException {
+    final String basePath = this.conf.getString(FlinkOptions.PATH);
+    final org.apache.hadoop.conf.Configuration hadoopConf = StreamerUtil.getHadoopConf();
+    // Hadoop FileSystem
+    try (FileSystem fs = FSUtils.getFs(basePath, hadoopConf)) {
+      if (!fs.exists(new Path(basePath, HoodieTableMetaClient.METAFOLDER_NAME))) {
+        HoodieTableMetaClient.initTableType(
+            hadoopConf,
+            basePath,
+            HoodieTableType.valueOf(this.conf.getString(FlinkOptions.TABLE_TYPE)),
+            this.conf.getString(FlinkOptions.TABLE_NAME),
+            "archived",
+            this.conf.getString(FlinkOptions.PAYLOAD_CLASS),
+            1);
+        LOG.info("Table initialized");
+      } else {
+        LOG.info("Table [{}/{}] already exists, no need to initialize the table",
+            basePath, this.conf.getString(FlinkOptions.TABLE_NAME));
+      }
+    }
+  }
+
+  static byte[] readBytes(DataInputStream in, int size) throws IOException {
+    byte[] bytes = new byte[size];
+    in.readFully(bytes);
+    return bytes;
+  }
+
+  /**
+   * Serialize the coordinator state. The current implementation may not be super efficient,
+   * but it should not matter that much because most of the state should be rather small.
+   * Large states themselves may already be a problem regardless of how the serialization
+   * is implemented.
+   *
+   * @return A byte array containing the serialized state of the source coordinator.
+   * @throws IOException When something goes wrong in serialization.
+   */
+  private byte[] writeCheckpointBytes() throws IOException {
+    try (ByteArrayOutputStream baos = new ByteArrayOutputStream();
+         DataOutputStream out = new DataOutputViewStreamWrapper(baos)) {
+
+      out.writeLong(this.inFlightCheckpoint);
+      byte[] serializedInstant = this.inFlightInstant.getBytes();
+      out.writeInt(serializedInstant.length);
+      out.write(serializedInstant);
+      out.flush();
+      return baos.toByteArray();
+    }
+  }
+
+  /**
+   * Restore the state of this source coordinator from the state bytes.
+   *
+   * @param bytes The checkpoint bytes that was returned from {@link #writeCheckpointBytes()}
+   * @throws Exception When the deserialization failed.
+   */
+  private void deserializeCheckpointAndRestore(byte[] bytes) throws Exception {
+    try (ByteArrayInputStream bais = new ByteArrayInputStream(bytes);
+         DataInputStream in = new DataInputViewStreamWrapper(bais)) {
+      long checkpointID = in.readLong();
+      int serializedInstantSize = in.readInt();
+      byte[] serializedInstant = readBytes(in, serializedInstantSize);
+      this.inFlightCheckpoint = checkpointID;
+      this.inFlightInstant = new String(serializedInstant);
+    }
+  }
+
+  private void reset() {
+    this.inFlightInstant = "";
+    this.eventBuffer = new BatchWriteSuccessEvent[this.parallelism];
+  }
+
+  private void checkAndForceCommit(String errMsg) {
+    if (!checkReady()) {
+      // forced but still has inflight instant
+      String inflightInstant = writeClient.getInflightAndRequestedInstant(this.conf.getString(FlinkOptions.TABLE_TYPE));
+      if (inflightInstant != null) {
+        assert inflightInstant.equals(this.inFlightInstant);
+        writeClient.rollback(this.inFlightInstant);
+        throw new HoodieException(errMsg);
+      }
+      if (Arrays.stream(eventBuffer).allMatch(Objects::isNull)) {
+        // The last checkpoint finished successfully.
+        return;
+      }
+    }
+    doCommit();
+  }
+
+  private void checkAndCommitWithRetry() {
+    int retryTimes = this.conf.getInteger(FlinkOptions.RETRY_TIMES);
+    long retryIntervalMillis = this.conf.getLong(FlinkOptions.RETRY_INTERVAL_MS);
+    int tryTimes = 0;
+    while (tryTimes++ < retryTimes) {
+      try {
+        if (!checkReady()) {
+          // Do not throw if the try times expires but the event buffer are still not ready,
+          // because we have a force check when next checkpoint starts.
+          waitFor(retryIntervalMillis);
+          continue;
+        }
+        doCommit();
+        return;
+      } catch (Throwable throwable) {
+        String cause = throwable.getCause() == null ? "" : throwable.getCause().toString();
+        LOG.warn("Try to commit the instant {} failed, with times {} and cause {}", this.inFlightInstant, tryTimes, cause);
+        if (tryTimes == retryTimes) {
+          throw new HoodieException(throwable);
+        }
+        waitFor(retryIntervalMillis);
+      }
+    }
+  }
+
+  private void waitFor(long intervalMillis) {

Review comment:
       `waitFor` what? Let us give it a more readable name?

##########
File path: hudi-flink/src/main/java/org/apache/hudi/operator/StreamWriteFunction.java
##########
@@ -0,0 +1,311 @@
+/*
+ * 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.keygen.KeyGenerator;
+import org.apache.hudi.operator.event.BatchWriteSuccessEvent;
+import org.apache.hudi.util.RowDataToAvroConverters;
+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.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.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 = StreamerUtil.getSourceSchema(config);
+    this.config = config;
+  }
+
+  @Override
+  public void open(Configuration parameters) throws IOException {
+    this.taskID = getRuntimeContext().getIndexOfThisSubtask();
+    this.keyGenerator = StreamerUtil.createKeyGenerator(FlinkOptions.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(FlinkOptions.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(toHoodieRecord(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(FlinkOptions.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);
+    }
+  }
+
+  /**
+   * 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 toHoodieRecord(I record) throws IOException {
+    boolean shouldCombine = this.config.getBoolean(FlinkOptions.INSERT_DROP_DUPS)
+        || WriteOperationType.fromValue(this.config.getString(FlinkOptions.OPERATION)) == WriteOperationType.UPSERT;
+    GenericRecord gr = (GenericRecord) this.converter.convert(this.avroSchema, record);
+    HoodieRecordPayload payload = shouldCombine
+        ? StreamerUtil.createPayload(this.config.getString(FlinkOptions.PAYLOAD_CLASS), gr,

Review comment:
       This method `createPayload` is too long and hard to read. Can you make it easy to read?




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



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

Posted by GitBox <gi...@apache.org>.
yanghua commented on a change in pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#discussion_r561720356



##########
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'd better find a way to unify the config names, e.g. do some refactor?




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



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

Posted by GitBox <gi...@apache.org>.
danny0405 commented on a change in pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#discussion_r560732699



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

Review comment:
       It is case sensitive.




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



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

Posted by GitBox <gi...@apache.org>.
danny0405 commented on a change in pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#discussion_r562365771



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




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



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

Posted by GitBox <gi...@apache.org>.
danny0405 commented on a change in pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#discussion_r562367018



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




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



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

Posted by GitBox <gi...@apache.org>.
danny0405 commented on a change in pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#discussion_r562371014



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




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



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

Posted by GitBox <gi...@apache.org>.
danny0405 commented on a change in pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#discussion_r565017180



##########
File path: hudi-flink/src/main/java/org/apache/hudi/operator/FlinkOptions.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.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");
+
+  // ------------------------------------------------------------------------
+  //  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")

Review comment:
       No, see `WriteOperationType#fromValue`.




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



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

Posted by GitBox <gi...@apache.org>.
danny0405 commented on a change in pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#discussion_r562375198



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



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

Posted by GitBox <gi...@apache.org>.
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



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

Posted by GitBox <gi...@apache.org>.
yanghua commented on a change in pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#discussion_r560727023



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

Review comment:
       Did you check if this config is case sensitive? That's to say if it works fine for `merge_on-read`.




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



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

Posted by GitBox <gi...@apache.org>.
yanghua commented on a change in pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#discussion_r560650566



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

Review comment:
       MUST we use the `Object` type here? No other choice?




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



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

Posted by GitBox <gi...@apache.org>.
danny0405 commented on a change in pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#discussion_r559434191



##########
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:
       checkpointId  is specified by the test cases, the test case needs to invoke the `checkpointFunction` by itself.




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



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

Posted by GitBox <gi...@apache.org>.
yanghua commented on a change in pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#discussion_r561716925



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

Review comment:
       So can we make `COPY_ON_WRITE` and `copy_on_write` equivalence?




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



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

Posted by GitBox <gi...@apache.org>.
yanghua commented on a change in pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#discussion_r559338335



##########
File path: hudi-flink/src/main/java/org/apache/hudi/streamer/Config.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 Config extends Configuration {

Review comment:
       IMO, this name is too generic. It would be better to make it more readable. What about renaming to `FlinkStreamerConfig`?




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



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

Posted by GitBox <gi...@apache.org>.
danny0405 commented on a change in pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#discussion_r556393067



##########
File path: hudi-common/src/main/java/org/apache/hudi/common/model/OverwriteWithLatestAvroPayload.java
##########
@@ -79,6 +79,11 @@ public OverwriteWithLatestAvroPayload preCombine(OverwriteWithLatestAvroPayload
    * @returns {@code true} if record represents a delete record. {@code false} otherwise.
    */
   protected boolean isDeleteRecord(GenericRecord genericRecord) {
+    final String isDeleteKey = "_hoodie_is_deleted";
+    // Modify to be compatible with old version Avro.
+    if (genericRecord.getSchema().getField(isDeleteKey) == null) {
+      return false;
+    }
     Object deleteMarker = genericRecord.get("_hoodie_is_deleted");

Review comment:
       We should fix the logic for Avro version that higher than 1.10, because the higher version throws directly if the field name does not exist in the schema.




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



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

Posted by GitBox <gi...@apache.org>.
danny0405 commented on a change in pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#discussion_r556383972



##########
File path: hudi-flink/src/main/java/org/apache/hudi/util/StreamerUtil.java
##########
@@ -81,16 +103,50 @@ public static DFSPropertiesConfiguration readConfig(FileSystem fs, Path cfgPath,
     return conf;
   }
 
-  public static Configuration getHadoopConf() {
-    return new Configuration();
+  public static org.apache.hadoop.conf.Configuration getHadoopConf() {
+    // create HiveConf from hadoop configuration with hadoop conf directory configured.
+    org.apache.hadoop.conf.Configuration hadoopConf = null;
+    for (String possibleHadoopConfPath : HadoopUtils.possibleHadoopConfPaths(new Configuration())) {

Review comment:
       We need to find the correct hadoop configuration to create correct hadoop filesystem, this is also useful for `HoodieFlinkEngineContext`.




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



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

Posted by GitBox <gi...@apache.org>.
danny0405 commented on a change in pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#discussion_r557012704



##########
File path: hudi-flink/src/main/java/org/apache/hudi/util/StreamerUtil.java
##########
@@ -81,16 +103,50 @@ public static DFSPropertiesConfiguration readConfig(FileSystem fs, Path cfgPath,
     return conf;
   }
 
-  public static Configuration getHadoopConf() {
-    return new Configuration();
+  public static org.apache.hadoop.conf.Configuration getHadoopConf() {
+    // create HiveConf from hadoop configuration with hadoop conf directory configured.
+    org.apache.hadoop.conf.Configuration hadoopConf = null;
+    for (String possibleHadoopConfPath : HadoopUtils.possibleHadoopConfPaths(new Configuration())) {

Review comment:
       The method firstly find the specified path `fs.hdfs.hadoopconf`, then directory `HADOOP_CONF_DIR` `HADOOP_HOME/conf` `HADOOP_HOME/etc/hadoop` from the system environment.
   
   Even if storage is separated from computing, the `FileSystem` we created is still correct, if we split the hadoop conf files correctly.
   
   In any case, we should not pass an empty hadoop configuration.




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



[GitHub] [hudi] codecov-io edited a comment on pull request #2430: [HUDI-1522] Add a new pipeline for Flink writer

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#issuecomment-757736411


   # [Codecov](https://codecov.io/gh/apache/hudi/pull/2430?src=pr&el=h1) Report
   > Merging [#2430](https://codecov.io/gh/apache/hudi/pull/2430?src=pr&el=desc) (b46310d) into [master](https://codecov.io/gh/apache/hudi/commit/7ce3ac778eb475bf23ffa31243dc0843ec7d089a?el=desc) (7ce3ac7) will **decrease** coverage by `41.08%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/hudi/pull/2430/graphs/tree.svg?width=650&height=150&src=pr&token=VTTXabwbs2)](https://codecov.io/gh/apache/hudi/pull/2430?src=pr&el=tree)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master   #2430       +/-   ##
   ============================================
   - Coverage     50.76%   9.68%   -41.09%     
   + Complexity     3063      48     -3015     
   ============================================
     Files           419      53      -366     
     Lines         18777    1930    -16847     
     Branches       1918     230     -1688     
   ============================================
   - Hits           9533     187     -9346     
   + Misses         8468    1730     -6738     
   + Partials        776      13      -763     
   ```
   
   | Flag | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | hudicli | `?` | `?` | |
   | hudiclient | `?` | `?` | |
   | hudicommon | `?` | `?` | |
   | hudiflink | `?` | `?` | |
   | hudihadoopmr | `?` | `?` | |
   | hudisparkdatasource | `?` | `?` | |
   | hudisync | `?` | `?` | |
   | huditimelineservice | `?` | `?` | |
   | hudiutilities | `9.68% <ø> (-59.75%)` | `0.00 <ø> (ø)` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/hudi/pull/2430?src=pr&el=tree) | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | [...va/org/apache/hudi/utilities/IdentitySplitter.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL0lkZW50aXR5U3BsaXR0ZXIuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-2.00%)` | |
   | [...va/org/apache/hudi/utilities/schema/SchemaSet.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NjaGVtYS9TY2hlbWFTZXQuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-3.00%)` | |
   | [...a/org/apache/hudi/utilities/sources/RowSource.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvUm93U291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | [.../org/apache/hudi/utilities/sources/AvroSource.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvQXZyb1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-1.00%)` | |
   | [.../org/apache/hudi/utilities/sources/JsonSource.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvblNvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-1.00%)` | |
   | [...rg/apache/hudi/utilities/sources/CsvDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvQ3N2REZTU291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-10.00%)` | |
   | [...g/apache/hudi/utilities/sources/JsonDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvbkRGU1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | [...apache/hudi/utilities/sources/JsonKafkaSource.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvbkthZmthU291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-6.00%)` | |
   | [...pache/hudi/utilities/sources/ParquetDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvUGFycXVldERGU1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-5.00%)` | |
   | [...lities/schema/SchemaProviderWithPostProcessor.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NjaGVtYS9TY2hlbWFQcm92aWRlcldpdGhQb3N0UHJvY2Vzc29yLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | ... and [386 more](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree-more) | |
   


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



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

Posted by GitBox <gi...@apache.org>.
danny0405 commented on a change in pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#discussion_r556386359



##########
File path: hudi-flink/src/main/java/org/apache/hudi/util/StreamerUtil.java
##########
@@ -81,16 +103,50 @@ public static DFSPropertiesConfiguration readConfig(FileSystem fs, Path cfgPath,
     return conf;
   }
 
-  public static Configuration getHadoopConf() {
-    return new Configuration();
+  public static org.apache.hadoop.conf.Configuration getHadoopConf() {
+    // create HiveConf from hadoop configuration with hadoop conf directory configured.
+    org.apache.hadoop.conf.Configuration hadoopConf = null;
+    for (String possibleHadoopConfPath : HadoopUtils.possibleHadoopConfPaths(new Configuration())) {
+      hadoopConf = getHadoopConfiguration(possibleHadoopConfPath);
+      if (hadoopConf != null) {
+        break;
+      }
+    }
+    if (hadoopConf == null) {
+      hadoopConf = new org.apache.hadoop.conf.Configuration();
+    }
+    return hadoopConf;
   }
 
-  public static void checkRequiredProperties(TypedProperties props, List<String> checkPropNames) {
-    checkPropNames.forEach(prop -> {
-      if (!props.containsKey(prop)) {
-        throw new HoodieNotSupportedException("Required property " + prop + " is missing");
+  /**
+   * Returns a new Hadoop Configuration object using the path to the hadoop conf configured.
+   *
+   * @param hadoopConfDir Hadoop conf directory path.
+   * @return A Hadoop configuration instance.
+   */
+  private static org.apache.hadoop.conf.Configuration getHadoopConfiguration(String hadoopConfDir) {
+    if (new File(hadoopConfDir).exists()) {

Review comment:
       Do you mean `java.nio.file.Files` ? I guess `new File(hadoopConfDir).exists()` is more simpler.




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



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

Posted by GitBox <gi...@apache.org>.
yanghua commented on a change in pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#discussion_r561848906



##########
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:
       `block` seems hard to understand. Any better word?

##########
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:
       The reason? why disable the operator chain? 

##########
File path: hudi-flink/src/main/java/org/apache/hudi/streamer/HoodieFlinkStreamer.java
##########
@@ -0,0 +1,108 @@
+/*
+ * 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.client.WriteStatus;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.operator.InstantGenerateOperator;
+import org.apache.hudi.operator.KeyedWriteProcessFunction;
+import org.apache.hudi.operator.KeyedWriteProcessOperator;
+import org.apache.hudi.sink.CommitSink;
+import org.apache.hudi.source.JsonStringToHoodieRecordMapFunction;
+import org.apache.hudi.util.StreamerUtil;
+
+import com.beust.jcommander.JCommander;
+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.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 java.util.List;
+import java.util.Objects;
+
+/**
+ * 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 HoodieFlinkStreamer {
+  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:
       ditto

##########
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:
       unnecessary comment for the constructor? it's easy to understand.

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

Review comment:
       typo: `even` -> `event`




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



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

Posted by GitBox <gi...@apache.org>.
danny0405 commented on a change in pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#discussion_r565021738



##########
File path: hudi-flink/src/main/java/org/apache/hudi/operator/StreamWriteOperatorCoordinator.java
##########
@@ -0,0 +1,413 @@
+/*
+ * 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(
+              String.format("Failed to checkpoint Instant %s for source %s",
+                  this.inFlightInstant, this.getClass().getSimpleName()), throwable));
+    }
+  }
+
+  @Override
+  public void checkpointComplete(long checkpointId) {
+    // start to commit the instant.
+    checkAndCommitWithRetry();
+  }
+
+  public void notifyCheckpointAborted(long checkpointId) {
+    Preconditions.checkState(inFlightCheckpoint == checkpointId,
+        "The aborted checkpoint should always be the last checkpoint");
+    checkAndForceCommit("The last checkpoint was aborted, roll back the last write and throw");
+  }
+
+  @Override
+  public void resetToCheckpoint(@Nullable byte[] checkpointData) throws Exception {
+    if (checkpointData != null) {
+      // restore when any checkpoint completed
+      deserializeCheckpointAndRestore(checkpointData);
+    }
+  }
+
+  @Override
+  public void handleEventFromOperator(int i, OperatorEvent operatorEvent) {
+    // no event to handle
+    Preconditions.checkState(operatorEvent instanceof BatchWriteSuccessEvent,
+        "The coordinator can only handle BatchWriteSuccessEvent");
+    BatchWriteSuccessEvent event = (BatchWriteSuccessEvent) operatorEvent;
+    Preconditions.checkState(event.getInstantTime().equals(this.inFlightInstant),
+        String.format("Receive an unexpected event for instant %s from task %d",
+            event.getInstantTime(), event.getTaskID()));
+    this.eventBuffer[event.getTaskID()] = event;
+  }
+
+  @Override
+  public void subtaskFailed(int i, @Nullable Throwable throwable) {
+    // no operation
+  }
+
+  // -------------------------------------------------------------------------
+  //  Utilities
+  // -------------------------------------------------------------------------
+
+  @SuppressWarnings("rawtypes")
+  private void initWriteClient() {
+    writeClient = new HoodieFlinkWriteClient(
+        new HoodieFlinkEngineContext(new FlinkTaskContextSupplier(null)),
+        StreamerUtil.getHoodieClientConfig(this.conf),
+        true);
+  }
+
+  private void initTable() throws IOException {
+    final String basePath = this.conf.getString(FlinkOptions.PATH);
+    final org.apache.hadoop.conf.Configuration hadoopConf = StreamerUtil.getHadoopConf();
+    // Hadoop FileSystem
+    try (FileSystem fs = FSUtils.getFs(basePath, hadoopConf)) {
+      if (!fs.exists(new Path(basePath, HoodieTableMetaClient.METAFOLDER_NAME))) {
+        HoodieTableMetaClient.initTableType(
+            hadoopConf,
+            basePath,
+            HoodieTableType.valueOf(this.conf.getString(FlinkOptions.TABLE_TYPE)),
+            this.conf.getString(FlinkOptions.TABLE_NAME),
+            "archived",
+            this.conf.getString(FlinkOptions.PAYLOAD_CLASS),
+            1);
+        LOG.info("Table initialized");
+      } else {
+        LOG.info("Table [{}/{}] already exists, no need to initialize the table",
+            basePath, this.conf.getString(FlinkOptions.TABLE_NAME));
+      }
+    }
+  }
+
+  static byte[] readBytes(DataInputStream in, int size) throws IOException {
+    byte[] bytes = new byte[size];
+    in.readFully(bytes);
+    return bytes;
+  }
+
+  /**
+   * Serialize the coordinator state. The current implementation may not be super efficient,
+   * but it should not matter that much because most of the state should be rather small.
+   * Large states themselves may already be a problem regardless of how the serialization
+   * is implemented.
+   *
+   * @return A byte array containing the serialized state of the source coordinator.
+   * @throws IOException When something goes wrong in serialization.
+   */
+  private byte[] writeCheckpointBytes() throws IOException {
+    try (ByteArrayOutputStream baos = new ByteArrayOutputStream();
+         DataOutputStream out = new DataOutputViewStreamWrapper(baos)) {
+
+      out.writeLong(this.inFlightCheckpoint);
+      byte[] serializedInstant = this.inFlightInstant.getBytes();
+      out.writeInt(serializedInstant.length);
+      out.write(serializedInstant);
+      out.flush();
+      return baos.toByteArray();
+    }
+  }
+
+  /**
+   * Restore the state of this source coordinator from the state bytes.
+   *
+   * @param bytes The checkpoint bytes that was returned from {@link #writeCheckpointBytes()}
+   * @throws Exception When the deserialization failed.
+   */
+  private void deserializeCheckpointAndRestore(byte[] bytes) throws Exception {
+    try (ByteArrayInputStream bais = new ByteArrayInputStream(bytes);
+         DataInputStream in = new DataInputViewStreamWrapper(bais)) {
+      long checkpointID = in.readLong();
+      int serializedInstantSize = in.readInt();
+      byte[] serializedInstant = readBytes(in, serializedInstantSize);
+      this.inFlightCheckpoint = checkpointID;
+      this.inFlightInstant = new String(serializedInstant);
+    }
+  }
+
+  private void reset() {
+    this.inFlightInstant = "";
+    this.eventBuffer = new BatchWriteSuccessEvent[this.parallelism];
+  }
+
+  private void checkAndForceCommit(String errMsg) {
+    if (!checkReady()) {
+      // forced but still has inflight instant
+      String inflightInstant = writeClient.getInflightAndRequestedInstant(this.conf.getString(FlinkOptions.TABLE_TYPE));
+      if (inflightInstant != null) {
+        assert inflightInstant.equals(this.inFlightInstant);
+        writeClient.rollback(this.inFlightInstant);
+        throw new HoodieException(errMsg);
+      }
+      if (Arrays.stream(eventBuffer).allMatch(Objects::isNull)) {
+        // The last checkpoint finished successfully.
+        return;
+      }
+    }
+    doCommit();
+  }
+
+  private void checkAndCommitWithRetry() {
+    int retryTimes = this.conf.getInteger(FlinkOptions.RETRY_TIMES);
+    long retryIntervalMillis = this.conf.getLong(FlinkOptions.RETRY_INTERVAL_MS);
+    int tryTimes = 0;
+    while (tryTimes++ < retryTimes) {
+      try {
+        if (!checkReady()) {
+          // Do not throw if the try times expires but the event buffer are still not ready,
+          // because we have a force check when next checkpoint starts.
+          waitFor(retryIntervalMillis);
+          continue;
+        }
+        doCommit();
+        return;
+      } catch (Throwable throwable) {
+        String cause = throwable.getCause() == null ? "" : throwable.getCause().toString();

Review comment:
       Not sure how many exception categories there ~, so we'd better keep the re-try because Flink is a streaming system, less failover is better, we can add the check for the exception categories in the future if it is needs, (for example, if there is a exception that does not deserve to retry, just fail fast.)
   
   The user can forbidden the retry by setting the retry times to a negative though ~




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



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

Posted by GitBox <gi...@apache.org>.
yanghua commented on a change in pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#discussion_r559335892



##########
File path: hudi-flink/pom.xml
##########
@@ -124,28 +124,77 @@
       <artifactId>kafka-clients</artifactId>
       <version>${kafka.version}</version>
     </dependency>
+    <dependency>
+      <groupId>org.apache.flink</groupId>
+      <artifactId>flink-hadoop-compatibility_${scala.binary.version}</artifactId>
+      <version>${flink.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.flink</groupId>
+      <artifactId>flink-avro</artifactId>
+      <version>${flink.version}</version>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.flink</groupId>
+      <artifactId>flink-json</artifactId>
+      <version>${flink.version}</version>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.flink</groupId>
+      <artifactId>flink-table-common</artifactId>
+      <version>${flink.version}</version>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.flink</groupId>
+      <artifactId>flink-table-runtime-blink_${scala.binary.version}</artifactId>
+      <version>${flink.version}</version>
+      <scope>provided</scope>
+    </dependency>
 
     <!-- Hadoop -->
     <dependency>
       <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-common</artifactId>
       <scope>compile</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>org.slf4j</groupId>
+          <artifactId>slf4j-log4j12</artifactId>
+        </exclusion>
+      </exclusions>
     </dependency>
     <dependency>
       <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-hdfs</artifactId>
       <scope>compile</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>org.slf4j</groupId>
+          <artifactId>slf4j-log4j12</artifactId>
+        </exclusion>
+      </exclusions>
     </dependency>
     <dependency>
       <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-auth</artifactId>
       <scope>compile</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>org.slf4j</groupId>
+          <artifactId>slf4j-log4j12</artifactId>
+        </exclusion>
+      </exclusions>
     </dependency>
 
     <!-- Avro -->
     <dependency>
       <groupId>org.apache.avro</groupId>
       <artifactId>avro</artifactId>
+      <!-- Override the version to be same with Flink avro -->
+      <version>1.10.0</version>

Review comment:
       Will we upgrade the version of avro for the whole project?




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



[GitHub] [hudi] codecov-io edited a comment on pull request #2430: [HUDI-1522] Add a new pipeline for Flink writer

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#issuecomment-757736411


   # [Codecov](https://codecov.io/gh/apache/hudi/pull/2430?src=pr&el=h1) Report
   > Merging [#2430](https://codecov.io/gh/apache/hudi/pull/2430?src=pr&el=desc) (db65e1e) into [master](https://codecov.io/gh/apache/hudi/commit/a38612b10f6ae04644519270f9b5eb631a77c148?el=desc) (a38612b) will **decrease** coverage by `41.00%`.
   > The diff coverage is `100.00%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/hudi/pull/2430/graphs/tree.svg?width=650&height=150&src=pr&token=VTTXabwbs2)](https://codecov.io/gh/apache/hudi/pull/2430?src=pr&el=tree)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master   #2430       +/-   ##
   ============================================
   - Coverage     50.69%   9.68%   -41.01%     
   + Complexity     3059      48     -3011     
   ============================================
     Files           419      53      -366     
     Lines         18810    1930    -16880     
     Branches       1924     230     -1694     
   ============================================
   - Hits           9535     187     -9348     
   + Misses         8498    1730     -6768     
   + Partials        777      13      -764     
   ```
   
   | Flag | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | hudicli | `?` | `?` | |
   | hudiclient | `?` | `?` | |
   | hudicommon | `?` | `?` | |
   | hudiflink | `?` | `?` | |
   | hudihadoopmr | `?` | `?` | |
   | hudisparkdatasource | `?` | `?` | |
   | hudisync | `?` | `?` | |
   | huditimelineservice | `?` | `?` | |
   | hudiutilities | `9.68% <100.00%> (-59.80%)` | `0.00 <1.00> (ø)` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/hudi/pull/2430?src=pr&el=tree) | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | [.../apache/hudi/utilities/HoodieSnapshotExporter.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL0hvb2RpZVNuYXBzaG90RXhwb3J0ZXIuamF2YQ==) | `83.62% <100.00%> (-5.18%)` | `28.00 <1.00> (ø)` | |
   | [...va/org/apache/hudi/utilities/IdentitySplitter.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL0lkZW50aXR5U3BsaXR0ZXIuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-2.00%)` | |
   | [...va/org/apache/hudi/utilities/schema/SchemaSet.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NjaGVtYS9TY2hlbWFTZXQuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-3.00%)` | |
   | [...a/org/apache/hudi/utilities/sources/RowSource.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvUm93U291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | [.../org/apache/hudi/utilities/sources/AvroSource.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvQXZyb1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-1.00%)` | |
   | [.../org/apache/hudi/utilities/sources/JsonSource.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvblNvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-1.00%)` | |
   | [...rg/apache/hudi/utilities/sources/CsvDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvQ3N2REZTU291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-10.00%)` | |
   | [...g/apache/hudi/utilities/sources/JsonDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvbkRGU1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | [...apache/hudi/utilities/sources/JsonKafkaSource.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvbkthZmthU291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-6.00%)` | |
   | [...pache/hudi/utilities/sources/ParquetDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvUGFycXVldERGU1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-5.00%)` | |
   | ... and [367 more](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree-more) | |
   


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



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

Posted by GitBox <gi...@apache.org>.
yanghua commented on a change in pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#discussion_r560728186



##########
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")
+      .stringType()
+      .defaultValue("ts")
+      .withDescription("Field used in preCombining before actual write. When two records have the same\n"
+          + "key value, we will pick the one with the largest value for the precombine field,\n"
+          + "determined by Object.compareTo(..)");
+
+  public static final ConfigOption<String> PAYLOAD_CLASS = ConfigOptions
+      .key("write.payload.class")
+      .stringType()
+      .defaultValue(OverwriteWithLatestAvroPayload.class.getName())
+      .withDescription("Payload class used. Override this, if you like to roll your own merge logic, when upserting/inserting.\n"
+          + "This will render any value set for the option in-effective");
+
+  /**
+   * Flag to indicate whether to drop duplicates upon insert.
+   * By default insert will accept duplicates, to gain extra performance.
+   */
+  public static final ConfigOption<Boolean> INSERT_DROP_DUPS = ConfigOptions
+      .key("write.insert.drop.duplicates")
+      .booleanType()
+      .defaultValue(false)
+      .withDescription("Flag to indicate whether to drop duplicates upon insert.\n"
+          + "By default insert will accept duplicates, to gain extra performance");
+
+  public static final ConfigOption<Integer> RETRY_TIMES = ConfigOptions
+      .key("write.retry.times")
+      .intType()
+      .defaultValue(3)
+      .withDescription("Flag to indicate how many times streaming job should retry for a failed checkpoint batch.\n"
+          + "By default 3");
+
+  public static final ConfigOption<Long> RETRY_INTERVAL_MS = ConfigOptions
+      .key("write.retry.interval.ms")
+      .longType()
+      .defaultValue(2000L)
+      .withDescription("Flag to indicate how long (by millisecond) before a retry should issued for failed checkpoint batch.\n"
+          + "By default 2000 and it will be doubled by every retry");
+
+  public static final ConfigOption<Boolean> IGNORE_FAILED_BATCH = ConfigOptions
+      .key("write.ignore.failed.batch")
+      .booleanType()
+      .defaultValue(true)
+      .withDescription("Flag to indicate whether to ignore any non exception error (e.g. writestatus error). within a checkpoint batch.\n"
+          + "By default true (in favor of streaming progressing over data integrity)");
+
+  public static final ConfigOption<String> RECORD_KEY_FIELD = ConfigOptions
+      .key(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY)
+      .stringType()
+      .defaultValue("uuid")
+      .withDescription("Record key field. Value to be used as the `recordKey` component of `HoodieKey`.\n"
+          + "Actual value will be obtained by invoking .toString() on the field value. Nested fields can be specified using "
+          + "the dot notation eg: `a.b.c`");
+
+  public static final ConfigOption<String> PARTITION_PATH_FIELD = ConfigOptions
+      .key(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY)
+      .stringType()
+      .defaultValue("partition-path")
+      .withDescription("Partition path field. Value to be used at the `partitionPath` component of `HoodieKey`.\n"
+          + "Actual value obtained by invoking .toString()");
+
+  public static final ConfigOption<String> KEYGEN_CLASS = ConfigOptions
+      .key("hoodie.datasource.write.keygenerator.class")

Review comment:
       Replace with `HoodieWriteConfig.KEYGENERATOR_CLASS_PROP`? It would be better to reuse the existed config key.




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



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

Posted by GitBox <gi...@apache.org>.
danny0405 commented on a change in pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#discussion_r565017180



##########
File path: hudi-flink/src/main/java/org/apache/hudi/operator/FlinkOptions.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.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");
+
+  // ------------------------------------------------------------------------
+  //  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")

Review comment:
       not now ~




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



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

Posted by GitBox <gi...@apache.org>.
danny0405 commented on a change in pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#discussion_r562366909



##########
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` ?




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



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

Posted by GitBox <gi...@apache.org>.
yanghua commented on a change in pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#discussion_r561725132



##########
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:
       Actually, do not know we really need to use `RowType`, only for convenience to integrate with `table/sql`?




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



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

Posted by GitBox <gi...@apache.org>.
yanghua commented on a change in pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#discussion_r556310823



##########
File path: hudi-flink/src/main/java/org/apache/hudi/streamer/OperationConverter.java
##########
@@ -0,0 +1,32 @@
+/*
+ * 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. */

Review comment:
       Can we keep the consistency to make the comment of class level align to the existing classes?

##########
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.HoodieOptions;
+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 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);
+    // 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();
+
+    if (cfg.flinkCheckPointPath != null) {
+      env.setStateBackend(new FsStateBackend(cfg.flinkCheckPointPath));
+    }
+
+    Properties kafkaProps = StreamerUtil.getKafkaProps(cfg);

Review comment:
       And can we extract the same logic both in v1 and v2?

##########
File path: hudi-flink/src/main/java/org/apache/hudi/util/StreamerUtil.java
##########
@@ -81,16 +103,50 @@ public static DFSPropertiesConfiguration readConfig(FileSystem fs, Path cfgPath,
     return conf;
   }
 
-  public static Configuration getHadoopConf() {
-    return new Configuration();
+  public static org.apache.hadoop.conf.Configuration getHadoopConf() {
+    // create HiveConf from hadoop configuration with hadoop conf directory configured.
+    org.apache.hadoop.conf.Configuration hadoopConf = null;
+    for (String possibleHadoopConfPath : HadoopUtils.possibleHadoopConfPaths(new Configuration())) {

Review comment:
       Do we really need to find a possible config file that we may not know its config information? Or only a new Object just enough?

##########
File path: hudi-flink/src/main/java/org/apache/hudi/util/StreamerUtil.java
##########
@@ -81,16 +103,50 @@ public static DFSPropertiesConfiguration readConfig(FileSystem fs, Path cfgPath,
     return conf;
   }
 
-  public static Configuration getHadoopConf() {
-    return new Configuration();
+  public static org.apache.hadoop.conf.Configuration getHadoopConf() {
+    // create HiveConf from hadoop configuration with hadoop conf directory configured.
+    org.apache.hadoop.conf.Configuration hadoopConf = null;
+    for (String possibleHadoopConfPath : HadoopUtils.possibleHadoopConfPaths(new Configuration())) {
+      hadoopConf = getHadoopConfiguration(possibleHadoopConfPath);
+      if (hadoopConf != null) {
+        break;
+      }
+    }
+    if (hadoopConf == null) {
+      hadoopConf = new org.apache.hadoop.conf.Configuration();
+    }
+    return hadoopConf;
   }
 
-  public static void checkRequiredProperties(TypedProperties props, List<String> checkPropNames) {
-    checkPropNames.forEach(prop -> {
-      if (!props.containsKey(prop)) {
-        throw new HoodieNotSupportedException("Required property " + prop + " is missing");
+  /**
+   * Returns a new Hadoop Configuration object using the path to the hadoop conf configured.
+   *
+   * @param hadoopConfDir Hadoop conf directory path.
+   * @return A Hadoop configuration instance.
+   */
+  private static org.apache.hadoop.conf.Configuration getHadoopConfiguration(String hadoopConfDir) {
+    if (new File(hadoopConfDir).exists()) {

Review comment:
       Can we use `Files.exists()` to test?

##########
File path: hudi-flink/src/main/java/org/apache/hudi/util/StreamerUtil.java
##########
@@ -122,21 +193,62 @@ public static HoodieRecordPayload createPayload(String payloadClass, GenericReco
     }
   }
 
-  public static HoodieWriteConfig getHoodieClientConfig(HoodieFlinkStreamer.Config cfg) {
-    FileSystem fs = FSUtils.getFs(cfg.targetBasePath, getHadoopConf());
+  /**
+   * Create a payload class via reflection, do not ordering/precombine value.
+   */
+  public static HoodieRecordPayload createPayload(String payloadClass, GenericRecord record)
+      throws IOException {
+    try {
+      return (HoodieRecordPayload) ReflectionUtils.loadClass(payloadClass,
+          new Class<?>[] {Option.class}, Option.of(record));
+    } catch (Throwable e) {
+      throw new IOException("Could not create payload for class: " + payloadClass, e);
+    }
+  }
+
+  public static HoodieWriteConfig getHoodieClientConfig(Configuration conf) {
     HoodieWriteConfig.Builder builder =
-        HoodieWriteConfig.newBuilder().withEngineType(EngineType.FLINK).withPath(cfg.targetBasePath).combineInput(cfg.filterDupes, true)
-            .withCompactionConfig(HoodieCompactionConfig.newBuilder().withPayloadClass(cfg.payloadClassName).build())
-            .withPayloadConfig(HoodiePayloadConfig.newBuilder().withPayloadOrderingField(cfg.sourceOrderingField)
-                .build())
-            .forTable(cfg.targetTableName)
-            .withAutoCommit(false)
-            .withProps(readConfig(fs, new Path(cfg.propsFilePath), cfg.configs)
-                .getConfig());
-
-    builder = builder.withSchema(new FilebasedSchemaProvider(getProps(cfg)).getTargetSchema().toString());
-    HoodieWriteConfig config = builder.build();
-    return config;
+            HoodieWriteConfig.newBuilder()
+                    .withEngineType(EngineType.FLINK)
+                    .withPath(conf.getString(HoodieOptions.PATH))
+                    .combineInput(conf.getBoolean(HoodieOptions.INSERT_DROP_DUPS), true)
+                    .withCompactionConfig(
+                        HoodieCompactionConfig.newBuilder()
+                            .withPayloadClass(conf.getString(HoodieOptions.PAYLOAD_CLASS))
+                            .build())
+                    .forTable(conf.getString(HoodieOptions.TABLE_NAME))
+                    .withAutoCommit(false)
+                    .withProps(flinkConf2TypedProperties(HoodieOptions.flatOptions(conf)));
+
+    builder = builder.withSchema(getSourceSchema(conf).toString());
+    return builder.build();
+  }
+
+  /**
+   * Converts the give {@link Configuration} to {@link TypedProperties}.
+   * The default values are also set up.
+   *
+   * @param conf The flink configuration
+   * @return a TypedProperties instance
+   */
+  public static TypedProperties flinkConf2TypedProperties(Configuration conf) {
+    Properties properties = new Properties();
+    // put all the set up options
+    conf.addAllToProperties(properties);
+    // put all the default options
+    for (ConfigOption<?> option : HoodieOptions.OPTIONAL_OPTIONS) {
+      if (!conf.contains(option)) {
+        properties.put(option.key(), option.defaultValue());
+      }
+    }
+    return new TypedProperties(properties);
   }
 
+  public static void checkRequiredProperties(TypedProperties props, List<String> checkPropNames) {
+    checkPropNames.forEach(prop -> {
+      if (!props.containsKey(prop)) {
+        throw new HoodieNotSupportedException("Required property " + prop + " is missing");

Review comment:
       Why does it throw `HoodieNotSupportedException `? it's an `illegal config option` case?

##########
File path: hudi-common/src/main/java/org/apache/hudi/common/model/OverwriteWithLatestAvroPayload.java
##########
@@ -79,6 +79,11 @@ public OverwriteWithLatestAvroPayload preCombine(OverwriteWithLatestAvroPayload
    * @returns {@code true} if record represents a delete record. {@code false} otherwise.
    */
   protected boolean isDeleteRecord(GenericRecord genericRecord) {
+    final String isDeleteKey = "_hoodie_is_deleted";
+    // Modify to be compatible with old version Avro.
+    if (genericRecord.getSchema().getField(isDeleteKey) == null) {
+      return false;
+    }
     Object deleteMarker = genericRecord.get("_hoodie_is_deleted");

Review comment:
       Why do not change the literal to the `_hoodie_is_deleted ` that you defined?

##########
File path: hudi-flink/pom.xml
##########
@@ -124,28 +124,77 @@
       <artifactId>kafka-clients</artifactId>
       <version>${kafka.version}</version>
     </dependency>
+    <dependency>
+      <groupId>org.apache.flink</groupId>
+      <artifactId>flink-hadoop-compatibility_${scala.binary.version}</artifactId>
+      <version>${flink.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.flink</groupId>
+      <artifactId>flink-avro</artifactId>
+      <version>${flink.version}</version>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.flink</groupId>
+      <artifactId>flink-json</artifactId>
+      <version>${flink.version}</version>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.flink</groupId>
+      <artifactId>flink-table-common</artifactId>
+      <version>${flink.version}</version>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.flink</groupId>
+      <artifactId>flink-table-runtime-blink_${scala.binary.version}</artifactId>

Review comment:
       I am thinking about one question: how to serve the users who only use DataStream API. Only define one `${flink.version}`? Here introduced some artifacts which only created in the high-level flink version.

##########
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:
       Does `HoodieFlinkConfigOptions` sounds better?

##########
File path: hudi-flink/src/main/java/org/apache/hudi/streamer/OperationConverter.java
##########
@@ -0,0 +1,32 @@
+/*
+ * 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. */

Review comment:
       There is a `OperationConverter` in `org.apache.hudi.utilities.deltastreamer` let us use single one.

##########
File path: hudi-flink/src/main/java/org/apache/hudi/util/StreamerUtil.java
##########
@@ -18,47 +18,69 @@
 
 package org.apache.hudi.util;
 
-import org.apache.hudi.HoodieFlinkStreamer;
+import org.apache.hudi.exception.HoodieNotSupportedException;
+import org.apache.hudi.keygen.SimpleAvroKeyGenerator;
+import org.apache.hudi.streamer.Config;
 import org.apache.hudi.common.config.DFSPropertiesConfiguration;
 import org.apache.hudi.common.config.TypedProperties;
-import org.apache.hudi.common.fs.FSUtils;
 import org.apache.hudi.common.engine.EngineType;
+import org.apache.hudi.common.fs.FSUtils;
 import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.common.util.Option;
 import org.apache.hudi.common.util.ReflectionUtils;
 import org.apache.hudi.config.HoodieCompactionConfig;
-import org.apache.hudi.config.HoodiePayloadConfig;
 import org.apache.hudi.config.HoodieWriteConfig;
 import org.apache.hudi.exception.HoodieIOException;
-import org.apache.hudi.exception.HoodieNotSupportedException;
 import org.apache.hudi.keygen.KeyGenerator;
-import org.apache.hudi.keygen.SimpleAvroKeyGenerator;
+import org.apache.hudi.operator.HoodieOptions;
 import org.apache.hudi.schema.FilebasedSchemaProvider;
 
+import org.apache.avro.Schema;
 import org.apache.avro.generic.GenericRecord;
-import org.apache.hadoop.conf.Configuration;
+import org.apache.flink.api.java.hadoop.mapred.utils.HadoopUtils;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.BufferedReader;
+import java.io.File;
 import java.io.IOException;
 import java.io.StringReader;
 import java.util.List;
+import java.util.Properties;
 
+/** Utilities for Flink stream read and write. */

Review comment:
       ditto

##########
File path: hudi-flink/pom.xml
##########
@@ -174,48 +229,34 @@
       <version>0.9.7</version>
     </dependency>
 
-    <!-- Junit Test Suite -->
-    <dependency>
-      <groupId>org.junit.jupiter</groupId>
-      <artifactId>junit-jupiter-api</artifactId>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.junit.jupiter</groupId>
-      <artifactId>junit-jupiter-engine</artifactId>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.junit.vintage</groupId>
-      <artifactId>junit-vintage-engine</artifactId>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.junit.jupiter</groupId>
-      <artifactId>junit-jupiter-params</artifactId>
-      <scope>test</scope>
-    </dependency>
+    <!-- test dependencies -->
     <dependency>
-      <groupId>org.mockito</groupId>
-      <artifactId>mockito-junit-jupiter</artifactId>
+      <groupId>org.apache.flink</groupId>

Review comment:
       Can you make the changes in this file cleaner?

##########
File path: hudi-common/src/main/java/org/apache/hudi/avro/HoodieAvroUtils.java
##########
@@ -217,7 +217,7 @@ public static String addMetadataColumnTypes(String hiveColumnTypes) {
 
   private static Schema initRecordKeySchema() {
     Schema.Field recordKeyField =
-        new Schema.Field(HoodieRecord.RECORD_KEY_METADATA_FIELD, METADATA_FIELD_SCHEMA, "", NullNode.getInstance());

Review comment:
       Other places also used `NullNode.getInstance()`, we need to keep consistency. Revert or change all.

##########
File path: hudi-common/src/main/avro/HoodieRollbackMetadata.avsc
##########
@@ -51,10 +51,10 @@
      /* overlaps with 'commitsRollback' field. Adding this to track action type for all the instants being rolled back. */
      {
        "name": "instantsRollback",
-       "default": null,
+       "default": [],
        "type": {
           "type": "array",
-          "default": null,
+          "default": [],
           "items": "HoodieInstantInfo"

Review comment:
       Will the change affect the version less than 1.10?




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



[GitHub] [hudi] codecov-io edited a comment on pull request #2430: [HUDI-1522] Add a new pipeline for Flink writer

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#issuecomment-757736411


   # [Codecov](https://codecov.io/gh/apache/hudi/pull/2430?src=pr&el=h1) Report
   > Merging [#2430](https://codecov.io/gh/apache/hudi/pull/2430?src=pr&el=desc) (3e3ef16) into [master](https://codecov.io/gh/apache/hudi/commit/7ce3ac778eb475bf23ffa31243dc0843ec7d089a?el=desc) (7ce3ac7) will **decrease** coverage by `41.08%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/hudi/pull/2430/graphs/tree.svg?width=650&height=150&src=pr&token=VTTXabwbs2)](https://codecov.io/gh/apache/hudi/pull/2430?src=pr&el=tree)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master   #2430       +/-   ##
   ============================================
   - Coverage     50.76%   9.68%   -41.09%     
   + Complexity     3063      48     -3015     
   ============================================
     Files           419      53      -366     
     Lines         18777    1930    -16847     
     Branches       1918     230     -1688     
   ============================================
   - Hits           9533     187     -9346     
   + Misses         8468    1730     -6738     
   + Partials        776      13      -763     
   ```
   
   | Flag | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | hudicli | `?` | `?` | |
   | hudiclient | `?` | `?` | |
   | hudicommon | `?` | `?` | |
   | hudiflink | `?` | `?` | |
   | hudihadoopmr | `?` | `?` | |
   | hudisparkdatasource | `?` | `?` | |
   | hudisync | `?` | `?` | |
   | huditimelineservice | `?` | `?` | |
   | hudiutilities | `9.68% <ø> (-59.75%)` | `0.00 <ø> (ø)` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/hudi/pull/2430?src=pr&el=tree) | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | [...va/org/apache/hudi/utilities/IdentitySplitter.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL0lkZW50aXR5U3BsaXR0ZXIuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-2.00%)` | |
   | [...va/org/apache/hudi/utilities/schema/SchemaSet.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NjaGVtYS9TY2hlbWFTZXQuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-3.00%)` | |
   | [...a/org/apache/hudi/utilities/sources/RowSource.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvUm93U291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | [.../org/apache/hudi/utilities/sources/AvroSource.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvQXZyb1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-1.00%)` | |
   | [.../org/apache/hudi/utilities/sources/JsonSource.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvblNvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-1.00%)` | |
   | [...rg/apache/hudi/utilities/sources/CsvDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvQ3N2REZTU291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-10.00%)` | |
   | [...g/apache/hudi/utilities/sources/JsonDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvbkRGU1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | [...apache/hudi/utilities/sources/JsonKafkaSource.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvbkthZmthU291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-6.00%)` | |
   | [...pache/hudi/utilities/sources/ParquetDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvUGFycXVldERGU1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-5.00%)` | |
   | [...lities/schema/SchemaProviderWithPostProcessor.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NjaGVtYS9TY2hlbWFQcm92aWRlcldpdGhQb3N0UHJvY2Vzc29yLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | ... and [386 more](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree-more) | |
   


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



[GitHub] [hudi] codecov-io edited a comment on pull request #2430: [HUDI-1522] Add a new pipeline for Flink writer

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#issuecomment-757736411






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



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

Posted by GitBox <gi...@apache.org>.
yanghua commented on a change in pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#discussion_r560695862



##########
File path: hudi-flink/src/main/java/org/apache/hudi/schema/FilebasedSchemaProvider.java
##########
@@ -43,16 +45,13 @@
     private static final String TARGET_SCHEMA_FILE_PROP = "hoodie.deltastreamer.schemaprovider.target.schema.file";

Review comment:
       Yes, but we can not always use a confused config option, here we are not `deltastreamer`. We can handle it later, can you file a jira issue to track it?




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



[GitHub] [hudi] codecov-io edited a comment on pull request #2430: [HUDI-1522] Add a new pipeline for Flink writer

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#issuecomment-757736411






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



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

Posted by GitBox <gi...@apache.org>.
danny0405 commented on a change in pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#discussion_r560729773



##########
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:
       The config options in `HoodieWriteConfig` are too verbose, i think there is no need to follow it completely. Spark `DataSourceOptions` did this too ~




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



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

Posted by GitBox <gi...@apache.org>.
danny0405 commented on a change in pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#discussion_r560742629



##########
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")
+      .stringType()
+      .defaultValue("ts")
+      .withDescription("Field used in preCombining before actual write. When two records have the same\n"
+          + "key value, we will pick the one with the largest value for the precombine field,\n"
+          + "determined by Object.compareTo(..)");
+
+  public static final ConfigOption<String> PAYLOAD_CLASS = ConfigOptions
+      .key("write.payload.class")
+      .stringType()
+      .defaultValue(OverwriteWithLatestAvroPayload.class.getName())
+      .withDescription("Payload class used. Override this, if you like to roll your own merge logic, when upserting/inserting.\n"
+          + "This will render any value set for the option in-effective");
+
+  /**
+   * Flag to indicate whether to drop duplicates upon insert.
+   * By default insert will accept duplicates, to gain extra performance.
+   */
+  public static final ConfigOption<Boolean> INSERT_DROP_DUPS = ConfigOptions
+      .key("write.insert.drop.duplicates")
+      .booleanType()
+      .defaultValue(false)
+      .withDescription("Flag to indicate whether to drop duplicates upon insert.\n"
+          + "By default insert will accept duplicates, to gain extra performance");
+
+  public static final ConfigOption<Integer> RETRY_TIMES = ConfigOptions
+      .key("write.retry.times")
+      .intType()
+      .defaultValue(3)
+      .withDescription("Flag to indicate how many times streaming job should retry for a failed checkpoint batch.\n"
+          + "By default 3");
+
+  public static final ConfigOption<Long> RETRY_INTERVAL_MS = ConfigOptions
+      .key("write.retry.interval.ms")
+      .longType()
+      .defaultValue(2000L)
+      .withDescription("Flag to indicate how long (by millisecond) before a retry should issued for failed checkpoint batch.\n"
+          + "By default 2000 and it will be doubled by every retry");
+
+  public static final ConfigOption<Boolean> IGNORE_FAILED_BATCH = ConfigOptions
+      .key("write.ignore.failed.batch")
+      .booleanType()
+      .defaultValue(true)
+      .withDescription("Flag to indicate whether to ignore any non exception error (e.g. writestatus error). within a checkpoint batch.\n"
+          + "By default true (in favor of streaming progressing over data integrity)");
+
+  public static final ConfigOption<String> RECORD_KEY_FIELD = ConfigOptions
+      .key(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY)
+      .stringType()
+      .defaultValue("uuid")
+      .withDescription("Record key field. Value to be used as the `recordKey` component of `HoodieKey`.\n"
+          + "Actual value will be obtained by invoking .toString() on the field value. Nested fields can be specified using "
+          + "the dot notation eg: `a.b.c`");
+
+  public static final ConfigOption<String> PARTITION_PATH_FIELD = ConfigOptions
+      .key(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY)
+      .stringType()
+      .defaultValue("partition-path")
+      .withDescription("Partition path field. Value to be used at the `partitionPath` component of `HoodieKey`.\n"
+          + "Actual value obtained by invoking .toString()");
+
+  public static final ConfigOption<String> KEYGEN_CLASS = ConfigOptions
+      .key("hoodie.datasource.write.keygenerator.class")
+      .stringType()
+      .defaultValue(SimpleAvroKeyGenerator.class.getName())
+      .withDescription("Key generator class, that implements will extract the key out of incoming record");
+
+  public static final ConfigOption<Integer> WRITE_TASK_PARALLELISM = ConfigOptions
+      .key("write.task.parallelism")
+      .intType()
+      .defaultValue(4)
+      .withDescription("Parallelism of tasks that do actual write, default is 4");
+
+  // -------------------------------------------------------------------------
+  //  Utilities
+  // -------------------------------------------------------------------------
+
+  // Remember to update the set when adding new options.
+  public static final List<ConfigOption<?>> OPTIONAL_OPTIONS = Arrays.asList(
+      TABLE_TYPE, OPERATION, PRECOMBINE_FIELD, PAYLOAD_CLASS, INSERT_DROP_DUPS, RETRY_TIMES,
+      RETRY_INTERVAL_MS, IGNORE_FAILED_BATCH, RECORD_KEY_FIELD, PARTITION_PATH_FIELD, KEYGEN_CLASS
+  );
+
+  // Prefix for Hoodie specific properties.
+  public static final String PROPERTIES_PREFIX = "properties.";

Review comment:
       It could, thanks ~




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



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

Posted by GitBox <gi...@apache.org>.
garyli1019 commented on pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#issuecomment-767994932


   @danny0405 sorry for the delay on review, I was super busy this week. The bloom index was merged to master, can we add the bloom index option to this PR as well?


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



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

Posted by GitBox <gi...@apache.org>.
yanghua commented on a change in pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#discussion_r556533133



##########
File path: hudi-flink/src/main/java/org/apache/hudi/util/StreamerUtil.java
##########
@@ -245,10 +247,9 @@ public static TypedProperties flinkConf2TypedProperties(Configuration conf) {
   }
 
   public static void checkRequiredProperties(TypedProperties props, List<String> checkPropNames) {
-    checkPropNames.forEach(prop -> {
-      if (!props.containsKey(prop)) {
-        throw new HoodieNotSupportedException("Required property " + prop + " is missing");
-      }
-    });
+    checkPropNames.forEach(prop ->
+        Preconditions.checkState(

Review comment:
       Why can not merge the next two lines into this line?




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



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

Posted by GitBox <gi...@apache.org>.
danny0405 commented on a change in pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#discussion_r557013794



##########
File path: hudi-flink/src/main/java/org/apache/hudi/util/StreamerUtil.java
##########
@@ -245,10 +247,9 @@ public static TypedProperties flinkConf2TypedProperties(Configuration conf) {
   }
 
   public static void checkRequiredProperties(TypedProperties props, List<String> checkPropNames) {
-    checkPropNames.forEach(prop -> {
-      if (!props.containsKey(prop)) {
-        throw new HoodieNotSupportedException("Required property " + prop + " is missing");
-      }
-    });
+    checkPropNames.forEach(prop ->
+        Preconditions.checkState(

Review comment:
       Personally i prefer shorter line code. Refactoring it if Hoodie thinks long line code is ok.




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



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

Posted by GitBox <gi...@apache.org>.
danny0405 commented on a change in pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#discussion_r559424858



##########
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:
       Agree, thanks.




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



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

Posted by GitBox <gi...@apache.org>.
yanghua commented on a change in pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#discussion_r562494679



##########
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:
       > 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.
   
   I mean if two write clients use the same basic data structure, there would be more logic that could be put into the abstraction.
   




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



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

Posted by GitBox <gi...@apache.org>.
yanghua commented on pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#issuecomment-768269863


   @danny0405 oh, CI failed... Please fix it before merging, 3ks.


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



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

Posted by GitBox <gi...@apache.org>.
danny0405 commented on a change in pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#discussion_r565017180



##########
File path: hudi-flink/src/main/java/org/apache/hudi/operator/FlinkOptions.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.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");
+
+  // ------------------------------------------------------------------------
+  //  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")

Review comment:
       not see `WriteOperationType#fromValue`.




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



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

Posted by GitBox <gi...@apache.org>.
yanghua commented on a change in pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#discussion_r557331716



##########
File path: hudi-flink/pom.xml
##########
@@ -124,28 +124,77 @@
       <artifactId>kafka-clients</artifactId>
       <version>${kafka.version}</version>
     </dependency>
+    <dependency>
+      <groupId>org.apache.flink</groupId>
+      <artifactId>flink-hadoop-compatibility_${scala.binary.version}</artifactId>
+      <version>${flink.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.flink</groupId>
+      <artifactId>flink-avro</artifactId>
+      <version>${flink.version}</version>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.flink</groupId>
+      <artifactId>flink-json</artifactId>
+      <version>${flink.version}</version>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.flink</groupId>
+      <artifactId>flink-table-common</artifactId>
+      <version>${flink.version}</version>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.flink</groupId>
+      <artifactId>flink-table-runtime-blink_${scala.binary.version}</artifactId>

Review comment:
       I mean if users only change the value of Flink version placeholder(`${flink.version}`), it will still cause the compile error, right? Maybe we need to use `<profile>` for the bundle module or another solution?




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



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

Posted by GitBox <gi...@apache.org>.
danny0405 commented on a change in pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#discussion_r560667817



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

Review comment:
       Because it is the last operator in the pipeline, there is no need to care about the output type.




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



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

Posted by GitBox <gi...@apache.org>.
danny0405 commented on a change in pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#discussion_r559421648



##########
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:
       Fine to add `Flink` to the class name.




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



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

Posted by GitBox <gi...@apache.org>.
yanghua commented on a change in pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#discussion_r560730373



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

Review comment:
       Never used config option? Add it when necessary?




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



[GitHub] [hudi] codecov-io edited a comment on pull request #2430: [HUDI-1522] Remove the single parallelism operator from the Flink writer

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on pull request #2430:
URL: https://github.com/apache/hudi/pull/2430#issuecomment-757736411


   # [Codecov](https://codecov.io/gh/apache/hudi/pull/2430?src=pr&el=h1) Report
   > Merging [#2430](https://codecov.io/gh/apache/hudi/pull/2430?src=pr&el=desc) (7961488) into [master](https://codecov.io/gh/apache/hudi/commit/7ce3ac778eb475bf23ffa31243dc0843ec7d089a?el=desc) (7ce3ac7) will **increase** coverage by `0.81%`.
   > The diff coverage is `59.79%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/hudi/pull/2430/graphs/tree.svg?width=650&height=150&src=pr&token=VTTXabwbs2)](https://codecov.io/gh/apache/hudi/pull/2430?src=pr&el=tree)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #2430      +/-   ##
   ============================================
   + Coverage     50.76%   51.58%   +0.81%     
   - Complexity     3063     3109      +46     
   ============================================
     Files           419      418       -1     
     Lines         18777    18975     +198     
     Branches       1918     1931      +13     
   ============================================
   + Hits           9533     9789     +256     
   + Misses         8468     8389      -79     
   - Partials        776      797      +21     
   ```
   
   | Flag | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | hudicli | `37.26% <ø> (ø)` | `0.00 <ø> (ø)` | |
   | hudiclient | `100.00% <ø> (ø)` | `0.00 <ø> (ø)` | |
   | hudicommon | `52.05% <33.33%> (-0.03%)` | `0.00 <2.00> (ø)` | |
   | hudiflink | `53.98% <59.95%> (+43.78%)` | `0.00 <56.00> (ø)` | |
   | hudihadoopmr | `33.06% <ø> (ø)` | `0.00 <ø> (ø)` | |
   | hudisparkdatasource | `66.07% <ø> (ø)` | `0.00 <ø> (ø)` | |
   | hudisync | `48.61% <ø> (ø)` | `0.00 <ø> (ø)` | |
   | huditimelineservice | `66.84% <ø> (ø)` | `0.00 <ø> (ø)` | |
   | hudiutilities | `69.48% <ø> (+0.05%)` | `0.00 <ø> (ø)` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/hudi/pull/2430?src=pr&el=tree) | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | [...ain/java/org/apache/hudi/avro/HoodieAvroUtils.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvYXZyby9Ib29kaWVBdnJvVXRpbHMuamF2YQ==) | `57.07% <ø> (-0.21%)` | `41.00 <0.00> (ø)` | |
   | [...main/java/org/apache/hudi/HoodieFlinkStreamer.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS1mbGluay9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvaHVkaS9Ib29kaWVGbGlua1N0cmVhbWVyLmphdmE=) | `0.00% <0.00%> (ø)` | `0.00 <0.00> (ø)` | |
   | [.../org/apache/hudi/operator/StreamWriteOperator.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS1mbGluay9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvaHVkaS9vcGVyYXRvci9TdHJlYW1Xcml0ZU9wZXJhdG9yLmphdmE=) | `0.00% <0.00%> (ø)` | `0.00 <0.00> (?)` | |
   | [...ache/hudi/operator/StreamWriteOperatorFactory.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS1mbGluay9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvaHVkaS9vcGVyYXRvci9TdHJlYW1Xcml0ZU9wZXJhdG9yRmFjdG9yeS5qYXZh) | `0.00% <0.00%> (ø)` | `0.00 <0.00> (?)` | |
   | [...i/common/model/OverwriteWithLatestAvroPayload.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL21vZGVsL092ZXJ3cml0ZVdpdGhMYXRlc3RBdnJvUGF5bG9hZC5qYXZh) | `60.00% <33.33%> (-4.71%)` | `10.00 <2.00> (ø)` | |
   | [...c/main/java/org/apache/hudi/util/StreamerUtil.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS1mbGluay9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvaHVkaS91dGlsL1N0cmVhbWVyVXRpbC5qYXZh) | `35.89% <47.36%> (+24.26%)` | `9.00 <8.00> (+6.00)` | |
   | [.../hudi/operator/StreamWriteOperatorCoordinator.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS1mbGluay9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvaHVkaS9vcGVyYXRvci9TdHJlYW1Xcml0ZU9wZXJhdG9yQ29vcmRpbmF0b3IuamF2YQ==) | `63.52% <63.52%> (ø)` | `25.00 <25.00> (?)` | |
   | [.../org/apache/hudi/operator/StreamWriteFunction.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS1mbGluay9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvaHVkaS9vcGVyYXRvci9TdHJlYW1Xcml0ZUZ1bmN0aW9uLmphdmE=) | `68.67% <68.67%> (ø)` | `14.00 <14.00> (?)` | |
   | [...n/java/org/apache/hudi/operator/HoodieOptions.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS1mbGluay9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvaHVkaS9vcGVyYXRvci9Ib29kaWVPcHRpb25zLmphdmE=) | `74.35% <74.35%> (ø)` | `3.00 <3.00> (?)` | |
   | [...he/hudi/operator/event/BatchWriteSuccessEvent.java](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree#diff-aHVkaS1mbGluay9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvaHVkaS9vcGVyYXRvci9ldmVudC9CYXRjaFdyaXRlU3VjY2Vzc0V2ZW50LmphdmE=) | `100.00% <100.00%> (ø)` | `4.00 <4.00> (?)` | |
   | ... and [12 more](https://codecov.io/gh/apache/hudi/pull/2430/diff?src=pr&el=tree-more) | |
   


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