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 2020/10/14 15:16:59 UTC

[GitHub] [hudi] wangxianghu opened a new pull request #2176: [HUDI-1327][WIP] Introduce base implementation of hudi-flink-client

wangxianghu opened a new pull request #2176:
URL: https://github.com/apache/hudi/pull/2176


   ## *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
   
   *Provide base implementation of hudi-flonk-client*
   
   ## 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] wangxianghu commented on a change in pull request #2176: [HUDI-1327] Introduce base implementation of hudi-flink-client

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



##########
File path: hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/FlinkTaskContextSupplier.java
##########
@@ -0,0 +1,65 @@
+/*
+ * 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.client;
+
+import org.apache.hudi.client.common.EngineProperty;
+import org.apache.hudi.client.common.TaskContextSupplier;
+import org.apache.hudi.common.util.Option;
+
+import org.apache.flink.api.common.functions.RuntimeContext;
+
+import java.util.function.Supplier;
+
+/**
+ * Flink task context supplier.
+ */
+public class FlinkTaskContextSupplier extends TaskContextSupplier {
+  private org.apache.flink.api.common.functions.RuntimeContext flinkRuntimeContext;
+
+  public FlinkTaskContextSupplier(RuntimeContext flinkRuntimeContext) {
+    this.flinkRuntimeContext = flinkRuntimeContext;
+  }
+
+  public RuntimeContext getFlinkRuntimeContext() {
+    return flinkRuntimeContext;
+  }
+
+  @Override
+  public Supplier<Integer> getPartitionIdSupplier() {
+    return () -> this.flinkRuntimeContext.getIndexOfThisSubtask();
+  }
+
+  @Override
+  public Supplier<Integer> getStageIdSupplier() {
+    // need to check again

Review comment:
       > // TODO need to check again?
   
   here flink has no concept of stage, so I used `NumberOfParallelSubtasks` not sure if it is suitable
   any idea ?




----------------------------------------------------------------
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 #2176: [HUDI-1327] Introduce base implementation of hudi-flink-client

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



##########
File path: hudi-flink-writer/src/main/java/org/apache/hudi/schema/FilebasedSchemaProvider.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.schema;
+
+import org.apache.avro.Schema;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.exception.HoodieIOException;
+import org.apache.hudi.util.StreamerUtil;
+
+import java.io.IOException;
+import java.util.Collections;
+
+/**
+ * A simple schema provider, that reads off files on DFS.
+ */
+public class FilebasedSchemaProvider extends SchemaProvider {

Review comment:
       > ah ok, I got confused by the class name. Should we move this to `TableSchemaResolver` then, so we have schema related stuff in one place. WDYT?
   
   Yes, keep schema related stuff together makes sense. 
   Maybe we can leave `TableSchemaResolver` where it is and move `SchemaProvider` stuff to `hudi-client-common` , because `TableSchemaResolver`  basically will not be modified in the future, while `SchemaProvider`  is expansion-oriented, more likely to add new child implementation, and more like client stuff.
   
   besides, this `SchemaProvider` comes from `hudi-utilities` module, and there are still lots of utils can be reused there, we can file a new ticket to do things like this together




----------------------------------------------------------------
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 #2176: [HUDI-1327] Introduce base implementation of hudi-flink-client

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



##########
File path: hudi-flink-writer/src/main/java/org/apache/hudi/HudiFlinkStreamer.java
##########
@@ -0,0 +1,164 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi;
+
+import com.beust.jcommander.IStringConverter;
+import com.beust.jcommander.JCommander;
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.ParameterException;
+import org.apache.flink.api.common.serialization.SimpleStringSchema;
+import org.apache.flink.api.common.typeinfo.TypeHint;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.state.filesystem.FsStateBackend;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload;
+import org.apache.hudi.constant.Operation;
+import org.apache.hudi.operator.InstantGenerateOperator;
+import org.apache.hudi.operator.KeyedWriteProcessFunction;
+import org.apache.hudi.operator.WriteProcessOperator;
+import org.apache.hudi.sink.CommitSink;
+import org.apache.hudi.source.KafkaJson2HoodieRecord;
+import org.apache.hudi.util.StreamerUtil;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Properties;
+
+public class HudiFlinkStreamer {
+  public static void main(String[] args) throws Exception {
+    StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+
+    final Config cfg = new Config();
+    JCommander cmd = new JCommander(cfg, null, args);
+    if (cfg.help || args.length == 0) {
+      cmd.usage();
+      System.exit(1);
+    }
+    env.enableCheckpointing(cfg.checkpointInterval);
+    env.getConfig().setGlobalJobParameters(cfg);
+    env.getCheckpointConfig().setMaxConcurrentCheckpoints(1);

Review comment:
       Please add a description here, why we must hard-code this config option.




----------------------------------------------------------------
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 pull request #2176: [HUDI-1327][WIP] Introduce base implementation of hudi-flink-client

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






----------------------------------------------------------------
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 #2176: [HUDI-1327] Introduce base implementation of hudi-flink-client

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



##########
File path: hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/HoodieFlinkWriteClient.java
##########
@@ -0,0 +1,227 @@
+/*
+ * 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.client;
+
+import org.apache.hudi.client.common.HoodieEngineContext;
+import org.apache.hudi.client.common.HoodieFlinkEngineContext;
+import org.apache.hudi.client.embedded.EmbeddedTimelineService;
+import org.apache.hudi.common.model.HoodieCommitMetadata;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.common.model.HoodieWriteStat;
+import org.apache.hudi.common.model.WriteOperationType;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.HoodieTableVersion;
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.common.table.timeline.HoodieTimeline;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.index.FlinkHoodieIndex;
+import org.apache.hudi.index.HoodieIndex;
+import org.apache.hudi.table.BulkInsertPartitioner;
+import org.apache.hudi.table.HoodieFlinkTable;
+import org.apache.hudi.table.HoodieTable;
+import org.apache.hudi.table.action.HoodieWriteMetadata;
+import org.apache.hudi.table.upgrade.FlinkUpgradeDowngrade;
+
+import com.codahale.metrics.Timer;
+import org.apache.hadoop.conf.Configuration;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+@SuppressWarnings("checkstyle:LineLength")
+public class HoodieFlinkWriteClient<T extends HoodieRecordPayload> extends
+    AbstractHoodieWriteClient<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> {
+
+  public HoodieFlinkWriteClient(HoodieEngineContext context, HoodieWriteConfig clientConfig) {
+    super(context, clientConfig);
+  }
+
+  public HoodieFlinkWriteClient(HoodieEngineContext context, HoodieWriteConfig writeConfig, boolean rollbackPending) {
+    super(context, writeConfig, rollbackPending);
+  }
+
+  public HoodieFlinkWriteClient(HoodieEngineContext context, HoodieWriteConfig writeConfig, boolean rollbackPending,
+                                Option<EmbeddedTimelineService> timelineService) {
+    super(context, writeConfig, rollbackPending, timelineService);
+  }
+
+  /**
+   * Complete changes performed at the given instantTime marker with specified action.
+   */
+  @Override
+  protected HoodieIndex<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> createIndex(HoodieWriteConfig writeConfig) {
+    return FlinkHoodieIndex.createIndex((HoodieFlinkEngineContext) context, config);
+  }
+
+  @Override
+  public boolean commit(String instantTime, List<WriteStatus> writeStatuses, Option<Map<String, String>> extraMetadata, String commitActionType, Map<String, List<String>> partitionToReplacedFileIds) {
+    List<HoodieWriteStat> writeStats = writeStatuses.parallelStream().map(WriteStatus::getStat).collect(Collectors.toList());
+    return commitStats(instantTime, writeStats, extraMetadata, commitActionType, partitionToReplacedFileIds);
+  }
+
+  @Override
+  protected HoodieTable<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> createTable(HoodieWriteConfig config, Configuration hadoopConf) {
+    return HoodieFlinkTable.create(config, (HoodieFlinkEngineContext) context);
+  }
+
+  @Override
+  public List<HoodieRecord<T>> filterExists(List<HoodieRecord<T>> hoodieRecords) {
+    // Create a Hoodie table which encapsulated the commits and files visible
+    HoodieFlinkTable<T> table = HoodieFlinkTable.create(config, (HoodieFlinkEngineContext) context);
+    Timer.Context indexTimer = metrics.getIndexCtx();
+    List<HoodieRecord<T>> recordsWithLocation = getIndex().tagLocation(hoodieRecords, context, table);
+    metrics.updateIndexMetrics(LOOKUP_STR, metrics.getDurationInMs(indexTimer == null ? 0L : indexTimer.stop()));
+    return recordsWithLocation.stream().filter(v1 -> !v1.isCurrentLocationKnown()).collect(Collectors.toList());
+  }
+
+  /**
+   * Main API to run bootstrap to hudi.
+   */
+  @Override
+  public void bootstrap(Option<Map<String, String>> extraMetadata) {
+    if (rollbackPending) {
+      rollBackInflightBootstrap();
+    }
+    getTableAndInitCtx(WriteOperationType.UPSERT, HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS).bootstrap(context, extraMetadata);
+  }

Review comment:
       > would be moved to super class ?
   
   my bad, this override is unnecessary, we can remove 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] garyli1019 commented on pull request #2176: [HUDI-1327][WIP] Introduce base implementation of hudi-flink-client

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


   Excited to this PR is coming, thanks for your contribution @wangxianghu !
   Should we split this task into smaller PRs? We could also distribute work to others. WDYT?


----------------------------------------------------------------
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 pull request #2176: [HUDI-1327] Introduce base implementation of hudi-flink-client

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


   > @wangxianghu thanks for opening the PR, with this PR, how would I start the flink job and verify E2E ?
   
   here is the main class `org.apache.hudi.HudiFlinkStreamer`


----------------------------------------------------------------
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 #2176: [HUDI-1327] Introduce base implementation of hudi-flink-client

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



##########
File path: hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/index/FlinkHoodieIndex.java
##########
@@ -0,0 +1,90 @@
+/*
+ * 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.index;
+
+import org.apache.hudi.ApiMaturityLevel;
+import org.apache.hudi.PublicAPIMethod;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.client.common.HoodieEngineContext;
+import org.apache.hudi.client.common.HoodieFlinkEngineContext;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.common.util.ReflectionUtils;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieIndexException;
+import org.apache.hudi.index.state.FlinkInMemoryStateIndex;
+import org.apache.hudi.table.HoodieTable;
+
+import java.util.List;
+
+/**
+ * Base flink implementation of {@link HoodieIndex}.
+ * @param <T> payload type
+ */
+public abstract class FlinkHoodieIndex<T extends HoodieRecordPayload> extends HoodieIndex<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> {
+  protected FlinkHoodieIndex(HoodieWriteConfig config) {
+    super(config);
+  }
+
+  public static FlinkHoodieIndex createIndex(HoodieFlinkEngineContext context, HoodieWriteConfig config) {
+    // first use index class config to create index.
+    if (!StringUtils.isNullOrEmpty(config.getIndexClass())) {
+      Object instance = ReflectionUtils.loadClass(config.getIndexClass(), config);
+      if (!(instance instanceof HoodieIndex)) {
+        throw new HoodieIndexException(config.getIndexClass() + " is not a subclass of HoodieIndex");
+      }
+      return (FlinkHoodieIndex) instance;
+    }
+    switch (config.getIndexType()) {
+      case HBASE:
+        // TODO
+        return null;
+      case INMEMORY:
+        return new FlinkInMemoryStateIndex<>(context, config);
+      case BLOOM:
+        // TODO
+        return null;
+      case GLOBAL_BLOOM:
+        // TODO
+        return null;
+      case SIMPLE:
+        // TODO
+        return null;
+      case GLOBAL_SIMPLE:
+        // TODO
+        return null;

Review comment:
       > should we implement some index here ?
   
   how about another pr to add other indexes? This pr we can focus on basic functions




----------------------------------------------------------------
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] vinothchandar commented on a change in pull request #2176: [HUDI-1327] Introduce base implementation of hudi-flink-client

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



##########
File path: hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/index/state/FlinkInMemoryStateIndex.java
##########
@@ -0,0 +1,135 @@
+/*
+ * 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.index.state;
+
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.client.common.HoodieEngineContext;
+import org.apache.hudi.client.common.HoodieFlinkEngineContext;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordLocation;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieIndexException;
+import org.apache.hudi.index.FlinkHoodieIndex;
+import org.apache.hudi.table.HoodieTable;
+
+import org.apache.flink.api.common.state.MapState;
+import org.apache.flink.api.common.state.MapStateDescriptor;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.util.List;
+
+/**
+ * Hoodie index implementation backed by flink state.
+ *
+ * @param <T> type of payload
+ */
+public class FlinkInMemoryStateIndex<T extends HoodieRecordPayload> extends FlinkHoodieIndex<T> {

Review comment:
       @wangxianghu sounds good. please file a JIRA if there is not one already.




----------------------------------------------------------------
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 pull request #2176: [HUDI-1327] Introduce base implementation of hudi-flink-client

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


   > More comments, as I try to understand the impl more.
   > Has this implementation been tested on a cluster for e.g? I ask coz I see that we use a `List<HoodieRecord>` for the client implementation. I was expecting to deal with some distributed data object like DataStream.
   
   `DataStream` is much different from `RDD`, it is difficult to use unified API for `DataStream`


----------------------------------------------------------------
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 #2176: [HUDI-1327] Introduce base implementation of hudi-flink-client

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


   # [Codecov](https://codecov.io/gh/apache/hudi/pull/2176?src=pr&el=h1) Report
   > Merging [#2176](https://codecov.io/gh/apache/hudi/pull/2176?src=pr&el=desc) (50a26bb) into [master](https://codecov.io/gh/apache/hudi/commit/0364498ae3be553f7c9fab78469283a6958da2ff?el=desc) (0364498) will **decrease** coverage by `0.01%`.
   > The diff coverage is `40.00%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/hudi/pull/2176/graphs/tree.svg?width=650&height=150&src=pr&token=VTTXabwbs2)](https://codecov.io/gh/apache/hudi/pull/2176?src=pr&el=tree)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #2176      +/-   ##
   ============================================
   - Coverage     53.53%   53.52%   -0.02%     
   + Complexity     2770     2769       -1     
   ============================================
     Files           348      348              
     Lines         16099    16110      +11     
     Branches       1642     1644       +2     
   ============================================
   + Hits           8619     8623       +4     
   - Misses         6785     6788       +3     
   - Partials        695      699       +4     
   ```
   
   | Flag | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | hudicli | `38.37% <ø> (ø)` | `0.00 <ø> (ø)` | |
   | hudiclient | `100.00% <ø> (ø)` | `0.00 <ø> (ø)` | |
   | hudicommon | `55.28% <40.00%> (-0.03%)` | `0.00 <0.00> (ø)` | |
   | hudihadoopmr | `32.94% <ø> (ø)` | `0.00 <ø> (ø)` | |
   | hudispark | `65.58% <ø> (ø)` | `0.00 <ø> (ø)` | |
   | huditimelineservice | `65.30% <ø> (ø)` | `0.00 <ø> (ø)` | |
   | hudiutilities | `70.09% <ø> (ø)` | `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/2176?src=pr&el=tree) | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | [...di/common/table/timeline/HoodieActiveTimeline.java](https://codecov.io/gh/apache/hudi/pull/2176/diff?src=pr&el=tree#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL3RhYmxlL3RpbWVsaW5lL0hvb2RpZUFjdGl2ZVRpbWVsaW5lLmphdmE=) | `73.76% <40.00%> (-0.37%)` | `41.00 <0.00> (ø)` | |
   | [...che/hudi/common/table/log/HoodieLogFileReader.java](https://codecov.io/gh/apache/hudi/pull/2176/diff?src=pr&el=tree#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL3RhYmxlL2xvZy9Ib29kaWVMb2dGaWxlUmVhZGVyLmphdmE=) | `71.06% <0.00%> (-2.48%)` | `23.00% <0.00%> (-2.00%)` | |
   | [...e/hudi/common/table/log/HoodieLogFormatWriter.java](https://codecov.io/gh/apache/hudi/pull/2176/diff?src=pr&el=tree#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL3RhYmxlL2xvZy9Ib29kaWVMb2dGb3JtYXRXcml0ZXIuamF2YQ==) | `78.81% <0.00%> (+0.24%)` | `23.00% <0.00%> (+1.00%)` | |
   


----------------------------------------------------------------
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] vinothchandar commented on pull request #2176: [HUDI-1327] Introduce base implementation of hudi-flink-client

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


   This is a huge step!  worth tweeting about (if none does, I ll do 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] vinothchandar commented on pull request #2176: [HUDI-1327] Introduce base implementation of hudi-flink-client

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


   @yanghua @wangxianghu folks, is this ready for review. Please let me know when I should start a review. 


----------------------------------------------------------------
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] leesf commented on a change in pull request #2176: [HUDI-1327] Introduce base implementation of hudi-flink-client

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



##########
File path: hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/index/state/FlinkInMemoryStateIndex.java
##########
@@ -0,0 +1,135 @@
+/*
+ * 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.index.state;
+
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.client.common.HoodieEngineContext;
+import org.apache.hudi.client.common.HoodieFlinkEngineContext;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordLocation;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieIndexException;
+import org.apache.hudi.index.FlinkHoodieIndex;
+import org.apache.hudi.table.HoodieTable;
+
+import org.apache.flink.api.common.state.MapState;
+import org.apache.flink.api.common.state.MapStateDescriptor;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.util.List;
+
+/**
+ * Hoodie index implementation backed by flink state.
+ *
+ * @param <T> type of payload
+ */
+public class FlinkInMemoryStateIndex<T extends HoodieRecordPayload> extends FlinkHoodieIndex<T> {
+
+  private static final Logger LOG = LogManager.getLogger(FlinkInMemoryStateIndex.class);
+  private MapState<HoodieKey, HoodieRecordLocation> mapState;
+
+  public FlinkInMemoryStateIndex(HoodieFlinkEngineContext context, HoodieWriteConfig config) {
+    super(config);
+    if (context.getRuntimeContext() != null) {
+      MapStateDescriptor<HoodieKey, HoodieRecordLocation> indexStateDesc =
+          new MapStateDescriptor<>("indexState", TypeInformation.of(HoodieKey.class), TypeInformation.of(HoodieRecordLocation.class));
+      if (context.getRuntimeContext() != null) {
+        mapState = context.getRuntimeContext().getMapState(indexStateDesc);
+      }
+    }
+  }
+
+  @Override
+  public List<HoodieRecord<T>> tagLocation(List<HoodieRecord<T>> records,
+                                           HoodieEngineContext context,
+                                           HoodieTable<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> hoodieTable) throws HoodieIndexException {
+    return context.map(records, record -> {
+      try {
+        if (mapState.contains(record.getKey())) {
+          record.unseal();
+          record.setCurrentLocation(mapState.get(record.getKey()));
+          record.seal();
+        }
+      } catch (Exception e) {
+        LOG.error(String.format("Tag record location failed, key = %s, %s", record.getRecordKey(), e.getMessage()));

Review comment:
       throw 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 merged pull request #2176: [HUDI-1327] Introduce base implementation of hudi-flink-client

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


   


----------------------------------------------------------------
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 removed a comment on pull request #2176: [HUDI-1327][WIP] Introduce base implementation of hudi-flink-client

Posted by GitBox <gi...@apache.org>.
wangxianghu removed a comment on pull request #2176:
URL: https://github.com/apache/hudi/pull/2176#issuecomment-720457811


   > Excited to this PR is coming, thanks for your contribution @wangxianghu !
   > Should we split this task into smaller PRs? We could also distribute work to others. WDYT?
   Thanks for your advice. there is no need now, I have finished it in another branch in my repository. this pr is blocked by https://github.com/apache/hudi/pull/2200, waiting for it now.
   
   I planed to split it into smaller sub-tasks, but it's hard to verify correctness for a single pr, So pushed a big one with only basic functions. 
   There will still be lots of work to do after this PR. I'll distribute the rest to others 
   
   


----------------------------------------------------------------
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 #2176: [HUDI-1327] Introduce base implementation of hudi-flink-client

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



##########
File path: hudi-flink-writer/src/main/java/org/apache/hudi/HudiFlinkStreamer.java
##########
@@ -0,0 +1,164 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi;
+
+import com.beust.jcommander.IStringConverter;
+import com.beust.jcommander.JCommander;
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.ParameterException;
+import org.apache.flink.api.common.serialization.SimpleStringSchema;
+import org.apache.flink.api.common.typeinfo.TypeHint;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.state.filesystem.FsStateBackend;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload;
+import org.apache.hudi.constant.Operation;
+import org.apache.hudi.operator.InstantGenerateOperator;
+import org.apache.hudi.operator.KeyedWriteProcessFunction;
+import org.apache.hudi.operator.WriteProcessOperator;
+import org.apache.hudi.sink.CommitSink;
+import org.apache.hudi.source.KafkaJson2HoodieRecord;
+import org.apache.hudi.util.StreamerUtil;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Properties;
+
+public class HudiFlinkStreamer {

Review comment:
       done




----------------------------------------------------------------
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 #2176: [HUDI-1327] Introduce base implementation of hudi-flink-client

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



##########
File path: pom.xml
##########
@@ -53,6 +53,8 @@
     <module>hudi-integ-test</module>
     <module>packaging/hudi-integ-test-bundle</module>
     <module>hudi-examples</module>
+    <module>hudi-flink-writer</module>

Review comment:
       > could we just call this `hudi-flink` instead of hudi-flink-writer. It gives us the change to also build the reading support into the same module/bundle?
   
   yes, good idea




----------------------------------------------------------------
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] leesf commented on a change in pull request #2176: [HUDI-1327] Introduce base implementation of hudi-flink-client

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



##########
File path: hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/HoodieFlinkWriteClient.java
##########
@@ -0,0 +1,227 @@
+/*
+ * 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.client;
+
+import org.apache.hudi.client.common.HoodieEngineContext;
+import org.apache.hudi.client.common.HoodieFlinkEngineContext;
+import org.apache.hudi.client.embedded.EmbeddedTimelineService;
+import org.apache.hudi.common.model.HoodieCommitMetadata;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.common.model.HoodieWriteStat;
+import org.apache.hudi.common.model.WriteOperationType;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.HoodieTableVersion;
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.common.table.timeline.HoodieTimeline;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.index.FlinkHoodieIndex;
+import org.apache.hudi.index.HoodieIndex;
+import org.apache.hudi.table.BulkInsertPartitioner;
+import org.apache.hudi.table.HoodieFlinkTable;
+import org.apache.hudi.table.HoodieTable;
+import org.apache.hudi.table.action.HoodieWriteMetadata;
+import org.apache.hudi.table.upgrade.FlinkUpgradeDowngrade;
+
+import com.codahale.metrics.Timer;
+import org.apache.hadoop.conf.Configuration;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+@SuppressWarnings("checkstyle:LineLength")
+public class HoodieFlinkWriteClient<T extends HoodieRecordPayload> extends
+    AbstractHoodieWriteClient<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> {
+
+  public HoodieFlinkWriteClient(HoodieEngineContext context, HoodieWriteConfig clientConfig) {
+    super(context, clientConfig);
+  }
+
+  public HoodieFlinkWriteClient(HoodieEngineContext context, HoodieWriteConfig writeConfig, boolean rollbackPending) {
+    super(context, writeConfig, rollbackPending);
+  }
+
+  public HoodieFlinkWriteClient(HoodieEngineContext context, HoodieWriteConfig writeConfig, boolean rollbackPending,
+                                Option<EmbeddedTimelineService> timelineService) {
+    super(context, writeConfig, rollbackPending, timelineService);
+  }
+
+  /**
+   * Complete changes performed at the given instantTime marker with specified action.
+   */
+  @Override
+  protected HoodieIndex<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> createIndex(HoodieWriteConfig writeConfig) {
+    return FlinkHoodieIndex.createIndex((HoodieFlinkEngineContext) context, config);
+  }
+
+  @Override
+  public boolean commit(String instantTime, List<WriteStatus> writeStatuses, Option<Map<String, String>> extraMetadata, String commitActionType, Map<String, List<String>> partitionToReplacedFileIds) {
+    List<HoodieWriteStat> writeStats = writeStatuses.parallelStream().map(WriteStatus::getStat).collect(Collectors.toList());
+    return commitStats(instantTime, writeStats, extraMetadata, commitActionType, partitionToReplacedFileIds);
+  }
+
+  @Override
+  protected HoodieTable<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> createTable(HoodieWriteConfig config, Configuration hadoopConf) {
+    return HoodieFlinkTable.create(config, (HoodieFlinkEngineContext) context);
+  }
+
+  @Override
+  public List<HoodieRecord<T>> filterExists(List<HoodieRecord<T>> hoodieRecords) {
+    // Create a Hoodie table which encapsulated the commits and files visible
+    HoodieFlinkTable<T> table = HoodieFlinkTable.create(config, (HoodieFlinkEngineContext) context);
+    Timer.Context indexTimer = metrics.getIndexCtx();
+    List<HoodieRecord<T>> recordsWithLocation = getIndex().tagLocation(hoodieRecords, context, table);
+    metrics.updateIndexMetrics(LOOKUP_STR, metrics.getDurationInMs(indexTimer == null ? 0L : indexTimer.stop()));
+    return recordsWithLocation.stream().filter(v1 -> !v1.isCurrentLocationKnown()).collect(Collectors.toList());
+  }
+
+  /**
+   * Main API to run bootstrap to hudi.
+   */
+  @Override
+  public void bootstrap(Option<Map<String, String>> extraMetadata) {
+    if (rollbackPending) {
+      rollBackInflightBootstrap();
+    }
+    getTableAndInitCtx(WriteOperationType.UPSERT, HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS).bootstrap(context, extraMetadata);
+  }
+
+  @Override
+  public List<WriteStatus> upsert(List<HoodieRecord<T>> records, String instantTime) {
+    HoodieTable<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> table =
+        getTableAndInitCtx(WriteOperationType.UPSERT, instantTime);
+    table.validateUpsertSchema();
+    setOperationType(WriteOperationType.UPSERT);
+    this.asyncCleanerService = AsyncCleanerService.startAsyncCleaningIfEnabled(this, instantTime);
+    HoodieWriteMetadata<List<WriteStatus>> result = table.upsert(context, instantTime, records);
+    if (result.getIndexLookupDuration().isPresent()) {
+      metrics.updateIndexMetrics(LOOKUP_STR, result.getIndexLookupDuration().get().toMillis());
+    }
+    return postWrite(result, instantTime, table);
+  }
+
+  @Override
+  public List<WriteStatus> upsertPreppedRecords(List<HoodieRecord<T>> preppedRecords, String instantTime) {
+    // TODO
+    return null;
+  }
+
+  @Override
+  public List<WriteStatus> insert(List<HoodieRecord<T>> records, String instantTime) {
+    HoodieTable<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> table =
+        getTableAndInitCtx(WriteOperationType.INSERT, instantTime);
+    table.validateUpsertSchema();
+    setOperationType(WriteOperationType.INSERT);
+    this.asyncCleanerService = AsyncCleanerService.startAsyncCleaningIfEnabled(this, instantTime);
+    HoodieWriteMetadata<List<WriteStatus>> result = table.insert(context, instantTime, records);
+    if (result.getIndexLookupDuration().isPresent()) {
+      metrics.updateIndexMetrics(LOOKUP_STR, result.getIndexLookupDuration().get().toMillis());
+    }
+    return postWrite(result, instantTime, table);
+  }
+
+  @Override
+  public List<WriteStatus> insertPreppedRecords(List<HoodieRecord<T>> preppedRecords, String instantTime) {
+    // TODO
+    return null;
+  }
+
+  @Override
+  public List<WriteStatus> bulkInsert(List<HoodieRecord<T>> records, String instantTime) {
+    // TODO
+    return null;
+  }
+
+  @Override
+  public List<WriteStatus> bulkInsert(List<HoodieRecord<T>> records, String instantTime, Option<BulkInsertPartitioner<List<HoodieRecord<T>>>> userDefinedBulkInsertPartitioner) {
+    // TODO
+    return null;
+  }
+
+  @Override
+  public List<WriteStatus> bulkInsertPreppedRecords(List<HoodieRecord<T>> preppedRecords, String instantTime, Option<BulkInsertPartitioner<List<HoodieRecord<T>>>> bulkInsertPartitioner) {
+    // TODO
+    return null;
+  }
+
+  @Override
+  public List<WriteStatus> delete(List<HoodieKey> keys, String instantTime) {
+    // TODO

Review comment:
       implement delete 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 #2176: [HUDI-1327] Introduce base implementation of hudi-flink-client

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



##########
File path: hudi-flink-writer/src/main/java/org/apache/hudi/HudiFlinkStreamer.java
##########
@@ -0,0 +1,164 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi;
+
+import com.beust.jcommander.IStringConverter;
+import com.beust.jcommander.JCommander;
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.ParameterException;
+import org.apache.flink.api.common.serialization.SimpleStringSchema;
+import org.apache.flink.api.common.typeinfo.TypeHint;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.state.filesystem.FsStateBackend;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload;
+import org.apache.hudi.constant.Operation;
+import org.apache.hudi.operator.InstantGenerateOperator;
+import org.apache.hudi.operator.KeyedWriteProcessFunction;
+import org.apache.hudi.operator.WriteProcessOperator;
+import org.apache.hudi.sink.CommitSink;
+import org.apache.hudi.source.KafkaJson2HoodieRecord;
+import org.apache.hudi.util.StreamerUtil;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Properties;
+
+public class HudiFlinkStreamer {

Review comment:
       Since this is the entry point, it would be better to add some description.




----------------------------------------------------------------
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 #2176: [HUDI-1327] Introduce base implementation of hudi-flink-client

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



##########
File path: hudi-flink-writer/src/main/java/org/apache/hudi/operator/InstantGenerateOperator.java
##########
@@ -0,0 +1,219 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.operator;
+
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hudi.HudiFlinkStreamer;
+import org.apache.hudi.client.FlinkTaskContextSupplier;
+import org.apache.hudi.client.HoodieFlinkWriteClient;
+import org.apache.hudi.client.common.HoodieFlinkEngineContext;
+import org.apache.hudi.client.common.TaskContextSupplier;
+import org.apache.hudi.common.config.SerializableConfiguration;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieTableType;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.timeline.HoodieTimeline;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.util.StreamerUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+
+/**
+ * Operator helps to generate globally unique instant, it must be executed in one parallelism. Before generate a new
+ * instant , {@link InstantGenerateOperator} will always check whether the last instant has completed. if it is
+ * completed, a new instant will be generated immediately, otherwise, wait and check the state of last instant until
+ * time out and throw an exception.
+ */
+public class InstantGenerateOperator extends AbstractStreamOperator<HoodieRecord> implements OneInputStreamOperator<HoodieRecord, HoodieRecord> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(InstantGenerateOperator.class);
+  public static final String NAME = "InstantGenerateOperator";
+
+  private HudiFlinkStreamer.Config cfg;
+  private HoodieFlinkWriteClient writeClient;
+  private SerializableConfiguration serializableHadoopConf;
+  private transient FileSystem fs;
+  private String latestInstant = "";
+  private List<String> latestInstantList = new ArrayList<>(1);
+  private transient ListState<String> latestInstantState;
+  private List<StreamRecord> bufferedRecords = new LinkedList();
+  private transient ListState<StreamRecord> recordsState;
+  private Integer commitTimeout;
+
+  @Override
+  public void processElement(StreamRecord<HoodieRecord> streamRecord) throws Exception {
+    if (streamRecord.getValue() != null) {
+      bufferedRecords.add(streamRecord);
+      output.collect(streamRecord);
+    }
+  }
+
+  @Override
+  public void open() throws Exception {
+    super.open();
+    // get configs from runtimeContext
+    cfg = (HudiFlinkStreamer.Config) getRuntimeContext().getExecutionConfig().getGlobalJobParameters();
+
+    // timeout
+    commitTimeout = Integer.valueOf(cfg.flinkCommitTimeout);
+
+    // hadoopConf
+    serializableHadoopConf = new SerializableConfiguration(StreamerUtil.getHadoopConf());
+
+    // Hadoop FileSystem
+    fs = FSUtils.getFs(cfg.targetBasePath, serializableHadoopConf.get());
+
+    TaskContextSupplier taskContextSupplier = new FlinkTaskContextSupplier(null);
+
+    // writeClient
+    writeClient = new HoodieFlinkWriteClient(new HoodieFlinkEngineContext(taskContextSupplier), StreamerUtil.getHoodieClientConfig(cfg));
+
+    // init table, create it if not exists.
+    initTable();
+  }
+
+  @Override
+  public void prepareSnapshotPreBarrier(long checkpointId) throws Exception {
+    super.prepareSnapshotPreBarrier(checkpointId);
+    // check whether the last instant is completed, if not, wait 10s and then throws an exception
+    if (!StringUtils.isNullOrEmpty(latestInstant)) {
+      doChecker();
+      // last instant completed, set it empty
+      latestInstant = "";
+    }
+
+    // no data no new instant
+    if (!bufferedRecords.isEmpty()) {
+      latestInstant = startNewInstant(checkpointId);
+    }
+  }
+
+  @Override
+  public void initializeState(StateInitializationContext context) throws Exception {
+    // instantState
+    ListStateDescriptor<String> latestInstantStateDescriptor = new ListStateDescriptor<String>("latestInstant", String.class);
+    latestInstantState = context.getOperatorStateStore().getListState(latestInstantStateDescriptor);
+
+    // recordState
+    ListStateDescriptor<StreamRecord> recordsStateDescriptor = new ListStateDescriptor<StreamRecord>("recordsState", StreamRecord.class);
+    recordsState = context.getOperatorStateStore().getListState(recordsStateDescriptor);
+
+    if (context.isRestored()) {
+      Iterator<String> latestInstantIterator = latestInstantState.get().iterator();
+      latestInstantIterator.forEachRemaining(x -> latestInstant = x);
+      LOG.info("InstantGenerateOperator initializeState get latestInstant [{}]", latestInstant);
+
+      Iterator<StreamRecord> recordIterator = recordsState.get().iterator();
+      bufferedRecords.clear();
+      recordIterator.forEachRemaining(x -> bufferedRecords.add(x));
+    }
+  }
+
+  @Override
+  public void snapshotState(StateSnapshotContext functionSnapshotContext) throws Exception {
+    if (latestInstantList.isEmpty()) {
+      latestInstantList.add(latestInstant);
+    } else {
+      latestInstantList.set(0, latestInstant);
+    }
+    latestInstantState.update(latestInstantList);
+    LOG.info("Update latest instant [{}]", latestInstant);
+
+    recordsState.update(bufferedRecords);
+    LOG.info("Update records state size = [{}]", bufferedRecords.size());
+    bufferedRecords.clear();
+  }
+
+  /**
+   * Create a new instant.
+   *
+   * @param checkpointId
+   */
+  private String startNewInstant(long checkpointId) {
+    String newTime = writeClient.startCommit();
+    LOG.info("create instant [{}], at checkpoint [{}]", newTime, checkpointId);
+    return newTime;
+  }
+
+  /**
+   * Check the status of last instant.
+   */
+  private void doChecker() throws InterruptedException {
+    // query the requested and inflight commit/deltacommit instants
+    String commitType = cfg.tableType.equals(HoodieTableType.COPY_ON_WRITE.name()) ? HoodieTimeline.COMMIT_ACTION : HoodieTimeline.DELTA_COMMIT_ACTION;
+    LOG.info("Query latest instant [{}]", latestInstant);
+    List<String> rollbackPendingCommits = writeClient.getInflightsAndRequestedInstants(commitType);
+    int tryTimes = 0;
+    while (tryTimes < commitTimeout) {
+      tryTimes++;
+      StringBuffer sb = new StringBuffer();
+      if (rollbackPendingCommits.contains(latestInstant)) {
+        //清空 sb
+        rollbackPendingCommits.forEach(x -> sb.append(x).append(","));
+
+        LOG.warn("Latest transaction [{}] is not completed! unCompleted transaction:[{}],try times [{}]", latestInstant, sb.toString(), tryTimes);
+
+        Thread.sleep(1000);
+        rollbackPendingCommits = writeClient.getInflightsAndRequestedInstants(commitType);
+      } else {
+        LOG.warn("Latest transaction [{}] is completed! Completed transaction, try times [{}]", latestInstant, tryTimes);
+        return;
+      }
+    }
+    throw new InterruptedException("Last instant costs more than ten second, stop task now");
+  }
+
+
+  /**
+   * Create table if not exists.
+   */
+  private void initTable() throws IOException {
+    if (!fs.exists(new Path(cfg.targetBasePath))) {

Review comment:
       > What's the result if the path exists? Adding a log warning message?
   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] wangxianghu commented on a change in pull request #2176: [HUDI-1327] Introduce base implementation of hudi-flink-client

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



##########
File path: hudi-flink-writer/src/main/java/org/apache/hudi/constant/Operation.java
##########
@@ -0,0 +1,23 @@
+/*
+ * 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.constant;
+
+public enum Operation {

Review comment:
       done




----------------------------------------------------------------
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 #2176: [HUDI-1327] Introduce base implementation of hudi-flink-client

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



##########
File path: hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/index/FlinkHoodieIndex.java
##########
@@ -0,0 +1,90 @@
+/*
+ * 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.index;
+
+import org.apache.hudi.ApiMaturityLevel;
+import org.apache.hudi.PublicAPIMethod;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.client.common.HoodieEngineContext;
+import org.apache.hudi.client.common.HoodieFlinkEngineContext;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.common.util.ReflectionUtils;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieIndexException;
+import org.apache.hudi.index.state.FlinkInMemoryStateIndex;
+import org.apache.hudi.table.HoodieTable;
+
+import java.util.List;
+
+/**
+ * Base flink implementation of {@link HoodieIndex}.
+ * @param <T> payload type
+ */
+public abstract class FlinkHoodieIndex<T extends HoodieRecordPayload> extends HoodieIndex<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> {
+  protected FlinkHoodieIndex(HoodieWriteConfig config) {
+    super(config);
+  }
+
+  public static FlinkHoodieIndex createIndex(HoodieFlinkEngineContext context, HoodieWriteConfig config) {
+    // first use index class config to create index.
+    if (!StringUtils.isNullOrEmpty(config.getIndexClass())) {
+      Object instance = ReflectionUtils.loadClass(config.getIndexClass(), config);
+      if (!(instance instanceof HoodieIndex)) {
+        throw new HoodieIndexException(config.getIndexClass() + " is not a subclass of HoodieIndex");
+      }
+      return (FlinkHoodieIndex) instance;
+    }
+    switch (config.getIndexType()) {
+      case HBASE:

Review comment:
       done




----------------------------------------------------------------
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 pull request #2176: [HUDI-1327] Introduce base implementation of hudi-flink-client

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


   > @yanghua @wangxianghu folks, is this ready for review. Please let me know when I should start a review.
   
   @vinothchandar yes, it is ready 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] leesf commented on a change in pull request #2176: [HUDI-1327] Introduce base implementation of hudi-flink-client

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



##########
File path: hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/HoodieFlinkWriteClient.java
##########
@@ -0,0 +1,227 @@
+/*
+ * 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.client;
+
+import org.apache.hudi.client.common.HoodieEngineContext;
+import org.apache.hudi.client.common.HoodieFlinkEngineContext;
+import org.apache.hudi.client.embedded.EmbeddedTimelineService;
+import org.apache.hudi.common.model.HoodieCommitMetadata;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.common.model.HoodieWriteStat;
+import org.apache.hudi.common.model.WriteOperationType;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.HoodieTableVersion;
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.common.table.timeline.HoodieTimeline;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.index.FlinkHoodieIndex;
+import org.apache.hudi.index.HoodieIndex;
+import org.apache.hudi.table.BulkInsertPartitioner;
+import org.apache.hudi.table.HoodieFlinkTable;
+import org.apache.hudi.table.HoodieTable;
+import org.apache.hudi.table.action.HoodieWriteMetadata;
+import org.apache.hudi.table.upgrade.FlinkUpgradeDowngrade;
+
+import com.codahale.metrics.Timer;
+import org.apache.hadoop.conf.Configuration;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+@SuppressWarnings("checkstyle:LineLength")
+public class HoodieFlinkWriteClient<T extends HoodieRecordPayload> extends
+    AbstractHoodieWriteClient<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> {
+
+  public HoodieFlinkWriteClient(HoodieEngineContext context, HoodieWriteConfig clientConfig) {
+    super(context, clientConfig);
+  }
+
+  public HoodieFlinkWriteClient(HoodieEngineContext context, HoodieWriteConfig writeConfig, boolean rollbackPending) {
+    super(context, writeConfig, rollbackPending);
+  }
+
+  public HoodieFlinkWriteClient(HoodieEngineContext context, HoodieWriteConfig writeConfig, boolean rollbackPending,
+                                Option<EmbeddedTimelineService> timelineService) {
+    super(context, writeConfig, rollbackPending, timelineService);
+  }
+
+  /**
+   * Complete changes performed at the given instantTime marker with specified action.
+   */
+  @Override
+  protected HoodieIndex<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> createIndex(HoodieWriteConfig writeConfig) {
+    return FlinkHoodieIndex.createIndex((HoodieFlinkEngineContext) context, config);
+  }
+
+  @Override
+  public boolean commit(String instantTime, List<WriteStatus> writeStatuses, Option<Map<String, String>> extraMetadata, String commitActionType, Map<String, List<String>> partitionToReplacedFileIds) {
+    List<HoodieWriteStat> writeStats = writeStatuses.parallelStream().map(WriteStatus::getStat).collect(Collectors.toList());
+    return commitStats(instantTime, writeStats, extraMetadata, commitActionType, partitionToReplacedFileIds);
+  }
+
+  @Override
+  protected HoodieTable<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> createTable(HoodieWriteConfig config, Configuration hadoopConf) {
+    return HoodieFlinkTable.create(config, (HoodieFlinkEngineContext) context);
+  }
+
+  @Override
+  public List<HoodieRecord<T>> filterExists(List<HoodieRecord<T>> hoodieRecords) {
+    // Create a Hoodie table which encapsulated the commits and files visible
+    HoodieFlinkTable<T> table = HoodieFlinkTable.create(config, (HoodieFlinkEngineContext) context);
+    Timer.Context indexTimer = metrics.getIndexCtx();
+    List<HoodieRecord<T>> recordsWithLocation = getIndex().tagLocation(hoodieRecords, context, table);
+    metrics.updateIndexMetrics(LOOKUP_STR, metrics.getDurationInMs(indexTimer == null ? 0L : indexTimer.stop()));
+    return recordsWithLocation.stream().filter(v1 -> !v1.isCurrentLocationKnown()).collect(Collectors.toList());
+  }
+
+  /**
+   * Main API to run bootstrap to hudi.
+   */
+  @Override
+  public void bootstrap(Option<Map<String, String>> extraMetadata) {
+    if (rollbackPending) {
+      rollBackInflightBootstrap();
+    }
+    getTableAndInitCtx(WriteOperationType.UPSERT, HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS).bootstrap(context, extraMetadata);
+  }

Review comment:
       would be moved to super 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] vinothchandar commented on a change in pull request #2176: [HUDI-1327] Introduce base implementation of hudi-flink-client

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



##########
File path: hudi-flink-writer/src/main/java/org/apache/hudi/constant/Operation.java
##########
@@ -0,0 +1,23 @@
+/*
+ * 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.constant;
+
+public enum Operation {

Review comment:
       can we reuse the WriteOperationType enum? 

##########
File path: hudi-flink-writer/src/main/java/org/apache/hudi/constant/Operation.java
##########
@@ -0,0 +1,23 @@
+/*
+ * 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.constant;
+
+public enum Operation {
+  UPSERT, INSERT

Review comment:
       can we reuse WriteOperationType, that already exists in hudi-common? 

##########
File path: hudi-flink-writer/src/main/java/org/apache/hudi/HudiFlinkStreamer.java
##########
@@ -0,0 +1,188 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi;
+
+import com.beust.jcommander.IStringConverter;
+import com.beust.jcommander.JCommander;
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.ParameterException;
+import org.apache.flink.api.common.serialization.SimpleStringSchema;
+import org.apache.flink.api.common.typeinfo.TypeHint;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.state.filesystem.FsStateBackend;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload;
+import org.apache.hudi.constant.Operation;
+import org.apache.hudi.operator.InstantGenerateOperator;
+import org.apache.hudi.operator.KeyedWriteProcessFunction;
+import org.apache.hudi.operator.KeyedWriteProcessOperator;
+import org.apache.hudi.sink.CommitSink;
+import org.apache.hudi.source.JsonStringToHoodieRecordMapFunction;
+import org.apache.hudi.util.StreamerUtil;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Objects;
+import java.util.Properties;
+
+/**
+ * An Utility which can incrementally consume data from Kafka and apply it to the target table.
+ * currently, it only support MOR table and insert, upsert operation.

Review comment:
       In the future, I guess we can do more refactoring and make stuff work end-end with deltastreamer as well . We can provide a migration path for users then. For now, this is a solid approach to get stuff working end to end. 

##########
File path: hudi-flink-writer/src/main/java/org/apache/hudi/HudiFlinkStreamer.java
##########
@@ -0,0 +1,188 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi;
+
+import com.beust.jcommander.IStringConverter;
+import com.beust.jcommander.JCommander;
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.ParameterException;
+import org.apache.flink.api.common.serialization.SimpleStringSchema;
+import org.apache.flink.api.common.typeinfo.TypeHint;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.state.filesystem.FsStateBackend;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload;
+import org.apache.hudi.constant.Operation;
+import org.apache.hudi.operator.InstantGenerateOperator;
+import org.apache.hudi.operator.KeyedWriteProcessFunction;
+import org.apache.hudi.operator.KeyedWriteProcessOperator;
+import org.apache.hudi.sink.CommitSink;
+import org.apache.hudi.source.JsonStringToHoodieRecordMapFunction;
+import org.apache.hudi.util.StreamerUtil;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Objects;
+import java.util.Properties;
+
+/**
+ * An Utility which can incrementally consume data from Kafka and apply it to the target table.
+ * currently, it only support MOR table and insert, upsert operation.

Review comment:
       can we make sure the program throws errors if using COW etc.

##########
File path: pom.xml
##########
@@ -53,6 +53,8 @@
     <module>hudi-integ-test</module>
     <module>packaging/hudi-integ-test-bundle</module>
     <module>hudi-examples</module>
+    <module>hudi-flink-writer</module>

Review comment:
       could we just call this `hudi-flink` instead of hudi-flink-writer. It gives us the change to also build the reading support into the same module/bundle?




----------------------------------------------------------------
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 #2176: [HUDI-1327] Introduce base implementation of hudi-flink-client

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



##########
File path: hudi-flink-writer/src/main/java/org/apache/hudi/HudiFlinkStreamer.java
##########
@@ -0,0 +1,188 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi;
+
+import com.beust.jcommander.IStringConverter;
+import com.beust.jcommander.JCommander;
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.ParameterException;
+import org.apache.flink.api.common.serialization.SimpleStringSchema;
+import org.apache.flink.api.common.typeinfo.TypeHint;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.state.filesystem.FsStateBackend;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload;
+import org.apache.hudi.constant.Operation;
+import org.apache.hudi.operator.InstantGenerateOperator;
+import org.apache.hudi.operator.KeyedWriteProcessFunction;
+import org.apache.hudi.operator.KeyedWriteProcessOperator;
+import org.apache.hudi.sink.CommitSink;
+import org.apache.hudi.source.JsonStringToHoodieRecordMapFunction;
+import org.apache.hudi.util.StreamerUtil;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Objects;
+import java.util.Properties;
+
+/**
+ * An Utility which can incrementally consume data from Kafka and apply it to the target table.
+ * currently, it only support MOR table and insert, upsert operation.

Review comment:
       > can we make sure the program throws errors if using COW etc.
   
   My bad. this pr support COW only. will support MOR in another pr




----------------------------------------------------------------
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] vinothchandar commented on a change in pull request #2176: [HUDI-1327] Introduce base implementation of hudi-flink-client

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



##########
File path: hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/index/state/FlinkInMemoryStateIndex.java
##########
@@ -0,0 +1,135 @@
+/*
+ * 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.index.state;
+
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.client.common.HoodieEngineContext;
+import org.apache.hudi.client.common.HoodieFlinkEngineContext;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordLocation;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieIndexException;
+import org.apache.hudi.index.FlinkHoodieIndex;
+import org.apache.hudi.table.HoodieTable;
+
+import org.apache.flink.api.common.state.MapState;
+import org.apache.flink.api.common.state.MapStateDescriptor;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.util.List;
+
+/**
+ * Hoodie index implementation backed by flink state.
+ *
+ * @param <T> type of payload
+ */
+public class FlinkInMemoryStateIndex<T extends HoodieRecordPayload> extends FlinkHoodieIndex<T> {

Review comment:
       is this always in memory? or can it be rocksDB backed as well? 

##########
File path: hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/index/FlinkHoodieIndex.java
##########
@@ -0,0 +1,90 @@
+/*
+ * 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.index;
+
+import org.apache.hudi.ApiMaturityLevel;
+import org.apache.hudi.PublicAPIMethod;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.client.common.HoodieEngineContext;
+import org.apache.hudi.client.common.HoodieFlinkEngineContext;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.common.util.ReflectionUtils;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieIndexException;
+import org.apache.hudi.index.state.FlinkInMemoryStateIndex;
+import org.apache.hudi.table.HoodieTable;
+
+import java.util.List;
+
+/**
+ * Base flink implementation of {@link HoodieIndex}.
+ * @param <T> payload type
+ */
+public abstract class FlinkHoodieIndex<T extends HoodieRecordPayload> extends HoodieIndex<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> {
+  protected FlinkHoodieIndex(HoodieWriteConfig config) {
+    super(config);
+  }
+
+  public static FlinkHoodieIndex createIndex(HoodieFlinkEngineContext context, HoodieWriteConfig config) {
+    // first use index class config to create index.
+    if (!StringUtils.isNullOrEmpty(config.getIndexClass())) {
+      Object instance = ReflectionUtils.loadClass(config.getIndexClass(), config);
+      if (!(instance instanceof HoodieIndex)) {
+        throw new HoodieIndexException(config.getIndexClass() + " is not a subclass of HoodieIndex");
+      }
+      return (FlinkHoodieIndex) instance;
+    }
+    switch (config.getIndexType()) {
+      case HBASE:

Review comment:
       throw errors here, instead of returning null? 

##########
File path: hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/HoodieFlinkWriteClient.java
##########
@@ -0,0 +1,219 @@
+/*
+ * 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.client;
+
+import org.apache.hudi.client.common.HoodieEngineContext;
+import org.apache.hudi.client.common.HoodieFlinkEngineContext;
+import org.apache.hudi.client.embedded.EmbeddedTimelineService;
+import org.apache.hudi.common.model.HoodieCommitMetadata;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.common.model.HoodieWriteStat;
+import org.apache.hudi.common.model.WriteOperationType;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.HoodieTableVersion;
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.common.table.timeline.HoodieTimeline;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.index.FlinkHoodieIndex;
+import org.apache.hudi.index.HoodieIndex;
+import org.apache.hudi.table.BulkInsertPartitioner;
+import org.apache.hudi.table.HoodieFlinkTable;
+import org.apache.hudi.table.HoodieTable;
+import org.apache.hudi.table.action.HoodieWriteMetadata;
+import org.apache.hudi.table.upgrade.FlinkUpgradeDowngrade;
+
+import com.codahale.metrics.Timer;
+import org.apache.hadoop.conf.Configuration;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+@SuppressWarnings("checkstyle:LineLength")
+public class HoodieFlinkWriteClient<T extends HoodieRecordPayload> extends
+    AbstractHoodieWriteClient<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> {
+
+  public HoodieFlinkWriteClient(HoodieEngineContext context, HoodieWriteConfig clientConfig) {
+    super(context, clientConfig);
+  }
+
+  public HoodieFlinkWriteClient(HoodieEngineContext context, HoodieWriteConfig writeConfig, boolean rollbackPending) {

Review comment:
       unused? 

##########
File path: hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/HoodieFlinkWriteClient.java
##########
@@ -0,0 +1,219 @@
+/*
+ * 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.client;
+
+import org.apache.hudi.client.common.HoodieEngineContext;
+import org.apache.hudi.client.common.HoodieFlinkEngineContext;
+import org.apache.hudi.client.embedded.EmbeddedTimelineService;
+import org.apache.hudi.common.model.HoodieCommitMetadata;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.common.model.HoodieWriteStat;
+import org.apache.hudi.common.model.WriteOperationType;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.HoodieTableVersion;
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.common.table.timeline.HoodieTimeline;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.index.FlinkHoodieIndex;
+import org.apache.hudi.index.HoodieIndex;
+import org.apache.hudi.table.BulkInsertPartitioner;
+import org.apache.hudi.table.HoodieFlinkTable;
+import org.apache.hudi.table.HoodieTable;
+import org.apache.hudi.table.action.HoodieWriteMetadata;
+import org.apache.hudi.table.upgrade.FlinkUpgradeDowngrade;
+
+import com.codahale.metrics.Timer;
+import org.apache.hadoop.conf.Configuration;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+@SuppressWarnings("checkstyle:LineLength")
+public class HoodieFlinkWriteClient<T extends HoodieRecordPayload> extends
+    AbstractHoodieWriteClient<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> {
+
+  public HoodieFlinkWriteClient(HoodieEngineContext context, HoodieWriteConfig clientConfig) {
+    super(context, clientConfig);
+  }
+
+  public HoodieFlinkWriteClient(HoodieEngineContext context, HoodieWriteConfig writeConfig, boolean rollbackPending) {
+    super(context, writeConfig, rollbackPending);
+  }
+
+  public HoodieFlinkWriteClient(HoodieEngineContext context, HoodieWriteConfig writeConfig, boolean rollbackPending,
+                                Option<EmbeddedTimelineService> timelineService) {
+    super(context, writeConfig, rollbackPending, timelineService);
+  }
+
+  /**
+   * Complete changes performed at the given instantTime marker with specified action.
+   */
+  @Override
+  protected HoodieIndex<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> createIndex(HoodieWriteConfig writeConfig) {
+    return FlinkHoodieIndex.createIndex((HoodieFlinkEngineContext) context, config);
+  }
+
+  @Override
+  public boolean commit(String instantTime, List<WriteStatus> writeStatuses, Option<Map<String, String>> extraMetadata, String commitActionType, Map<String, List<String>> partitionToReplacedFileIds) {
+    List<HoodieWriteStat> writeStats = writeStatuses.parallelStream().map(WriteStatus::getStat).collect(Collectors.toList());
+    return commitStats(instantTime, writeStats, extraMetadata, commitActionType, partitionToReplacedFileIds);
+  }
+
+  @Override
+  protected HoodieTable<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> createTable(HoodieWriteConfig config, Configuration hadoopConf) {
+    return HoodieFlinkTable.create(config, (HoodieFlinkEngineContext) context);
+  }
+
+  @Override
+  public List<HoodieRecord<T>> filterExists(List<HoodieRecord<T>> hoodieRecords) {
+    // Create a Hoodie table which encapsulated the commits and files visible
+    HoodieFlinkTable<T> table = HoodieFlinkTable.create(config, (HoodieFlinkEngineContext) context);
+    Timer.Context indexTimer = metrics.getIndexCtx();
+    List<HoodieRecord<T>> recordsWithLocation = getIndex().tagLocation(hoodieRecords, context, table);
+    metrics.updateIndexMetrics(LOOKUP_STR, metrics.getDurationInMs(indexTimer == null ? 0L : indexTimer.stop()));
+    return recordsWithLocation.stream().filter(v1 -> !v1.isCurrentLocationKnown()).collect(Collectors.toList());
+  }
+
+  @Override
+  public List<WriteStatus> upsert(List<HoodieRecord<T>> records, String instantTime) {
+    HoodieTable<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> table =
+        getTableAndInitCtx(WriteOperationType.UPSERT, instantTime);
+    table.validateUpsertSchema();
+    setOperationType(WriteOperationType.UPSERT);
+    this.asyncCleanerService = AsyncCleanerService.startAsyncCleaningIfEnabled(this, instantTime);
+    HoodieWriteMetadata<List<WriteStatus>> result = table.upsert(context, instantTime, records);
+    if (result.getIndexLookupDuration().isPresent()) {
+      metrics.updateIndexMetrics(LOOKUP_STR, result.getIndexLookupDuration().get().toMillis());
+    }
+    return postWrite(result, instantTime, table);
+  }
+
+  @Override
+  public List<WriteStatus> upsertPreppedRecords(List<HoodieRecord<T>> preppedRecords, String instantTime) {
+    // TODO
+    return null;
+  }
+
+  @Override
+  public List<WriteStatus> insert(List<HoodieRecord<T>> records, String instantTime) {
+    HoodieTable<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> table =
+        getTableAndInitCtx(WriteOperationType.INSERT, instantTime);
+    table.validateUpsertSchema();
+    setOperationType(WriteOperationType.INSERT);
+    this.asyncCleanerService = AsyncCleanerService.startAsyncCleaningIfEnabled(this, instantTime);
+    HoodieWriteMetadata<List<WriteStatus>> result = table.insert(context, instantTime, records);
+    if (result.getIndexLookupDuration().isPresent()) {
+      metrics.updateIndexMetrics(LOOKUP_STR, result.getIndexLookupDuration().get().toMillis());
+    }
+    return postWrite(result, instantTime, table);
+  }
+
+  @Override
+  public List<WriteStatus> insertPreppedRecords(List<HoodieRecord<T>> preppedRecords, String instantTime) {
+    // TODO
+    return null;

Review comment:
       same comment. can we throw an Unsupported exception here instead of returning null. 




----------------------------------------------------------------
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 #2176: [HUDI-1327] Introduce base implementation of hudi-flink-client

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



##########
File path: hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/HoodieFlinkWriteClient.java
##########
@@ -0,0 +1,219 @@
+/*
+ * 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.client;
+
+import org.apache.hudi.client.common.HoodieEngineContext;
+import org.apache.hudi.client.common.HoodieFlinkEngineContext;
+import org.apache.hudi.client.embedded.EmbeddedTimelineService;
+import org.apache.hudi.common.model.HoodieCommitMetadata;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.common.model.HoodieWriteStat;
+import org.apache.hudi.common.model.WriteOperationType;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.HoodieTableVersion;
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.common.table.timeline.HoodieTimeline;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.index.FlinkHoodieIndex;
+import org.apache.hudi.index.HoodieIndex;
+import org.apache.hudi.table.BulkInsertPartitioner;
+import org.apache.hudi.table.HoodieFlinkTable;
+import org.apache.hudi.table.HoodieTable;
+import org.apache.hudi.table.action.HoodieWriteMetadata;
+import org.apache.hudi.table.upgrade.FlinkUpgradeDowngrade;
+
+import com.codahale.metrics.Timer;
+import org.apache.hadoop.conf.Configuration;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+@SuppressWarnings("checkstyle:LineLength")
+public class HoodieFlinkWriteClient<T extends HoodieRecordPayload> extends
+    AbstractHoodieWriteClient<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> {
+
+  public HoodieFlinkWriteClient(HoodieEngineContext context, HoodieWriteConfig clientConfig) {
+    super(context, clientConfig);
+  }
+
+  public HoodieFlinkWriteClient(HoodieEngineContext context, HoodieWriteConfig writeConfig, boolean rollbackPending) {
+    super(context, writeConfig, rollbackPending);
+  }
+
+  public HoodieFlinkWriteClient(HoodieEngineContext context, HoodieWriteConfig writeConfig, boolean rollbackPending,
+                                Option<EmbeddedTimelineService> timelineService) {
+    super(context, writeConfig, rollbackPending, timelineService);
+  }
+
+  /**
+   * Complete changes performed at the given instantTime marker with specified action.
+   */
+  @Override
+  protected HoodieIndex<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> createIndex(HoodieWriteConfig writeConfig) {
+    return FlinkHoodieIndex.createIndex((HoodieFlinkEngineContext) context, config);
+  }
+
+  @Override
+  public boolean commit(String instantTime, List<WriteStatus> writeStatuses, Option<Map<String, String>> extraMetadata, String commitActionType, Map<String, List<String>> partitionToReplacedFileIds) {
+    List<HoodieWriteStat> writeStats = writeStatuses.parallelStream().map(WriteStatus::getStat).collect(Collectors.toList());
+    return commitStats(instantTime, writeStats, extraMetadata, commitActionType, partitionToReplacedFileIds);
+  }
+
+  @Override
+  protected HoodieTable<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> createTable(HoodieWriteConfig config, Configuration hadoopConf) {
+    return HoodieFlinkTable.create(config, (HoodieFlinkEngineContext) context);
+  }
+
+  @Override
+  public List<HoodieRecord<T>> filterExists(List<HoodieRecord<T>> hoodieRecords) {
+    // Create a Hoodie table which encapsulated the commits and files visible
+    HoodieFlinkTable<T> table = HoodieFlinkTable.create(config, (HoodieFlinkEngineContext) context);
+    Timer.Context indexTimer = metrics.getIndexCtx();
+    List<HoodieRecord<T>> recordsWithLocation = getIndex().tagLocation(hoodieRecords, context, table);
+    metrics.updateIndexMetrics(LOOKUP_STR, metrics.getDurationInMs(indexTimer == null ? 0L : indexTimer.stop()));
+    return recordsWithLocation.stream().filter(v1 -> !v1.isCurrentLocationKnown()).collect(Collectors.toList());
+  }
+
+  @Override
+  public List<WriteStatus> upsert(List<HoodieRecord<T>> records, String instantTime) {
+    HoodieTable<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> table =
+        getTableAndInitCtx(WriteOperationType.UPSERT, instantTime);
+    table.validateUpsertSchema();
+    setOperationType(WriteOperationType.UPSERT);
+    this.asyncCleanerService = AsyncCleanerService.startAsyncCleaningIfEnabled(this, instantTime);
+    HoodieWriteMetadata<List<WriteStatus>> result = table.upsert(context, instantTime, records);
+    if (result.getIndexLookupDuration().isPresent()) {
+      metrics.updateIndexMetrics(LOOKUP_STR, result.getIndexLookupDuration().get().toMillis());
+    }
+    return postWrite(result, instantTime, table);
+  }
+
+  @Override
+  public List<WriteStatus> upsertPreppedRecords(List<HoodieRecord<T>> preppedRecords, String instantTime) {
+    // TODO
+    return null;
+  }
+
+  @Override
+  public List<WriteStatus> insert(List<HoodieRecord<T>> records, String instantTime) {
+    HoodieTable<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> table =
+        getTableAndInitCtx(WriteOperationType.INSERT, instantTime);
+    table.validateUpsertSchema();
+    setOperationType(WriteOperationType.INSERT);
+    this.asyncCleanerService = AsyncCleanerService.startAsyncCleaningIfEnabled(this, instantTime);
+    HoodieWriteMetadata<List<WriteStatus>> result = table.insert(context, instantTime, records);
+    if (result.getIndexLookupDuration().isPresent()) {
+      metrics.updateIndexMetrics(LOOKUP_STR, result.getIndexLookupDuration().get().toMillis());
+    }
+    return postWrite(result, instantTime, table);
+  }
+
+  @Override
+  public List<WriteStatus> insertPreppedRecords(List<HoodieRecord<T>> preppedRecords, String instantTime) {
+    // TODO
+    return null;

Review comment:
       > same comment. can we throw an Unsupported exception here instead of returning null.
   
   sure, I'll fix them  so as the `index`




----------------------------------------------------------------
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 pull request #2176: [HUDI-1327] Introduce base implementation of hudi-flink-client

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


   > More comments, as I try to understand the impl more.
   > Has this implementation been tested on a cluster for e.g? I ask coz I see that we use a `List<HoodieRecord>` for the client implementation. I was expecting to deal with some distributed data object like DataStream.
   
   Yes, I tested it in our dev cluster, it looks 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 commented on pull request #2176: [HUDI-1327] Introduce base implementation of hudi-flink-client

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


   # [Codecov](https://codecov.io/gh/apache/hudi/pull/2176?src=pr&el=h1) Report
   > Merging [#2176](https://codecov.io/gh/apache/hudi/pull/2176?src=pr&el=desc) (07097e7) into [master](https://codecov.io/gh/apache/hudi/commit/0364498ae3be553f7c9fab78469283a6958da2ff?el=desc) (0364498) will **increase** coverage by `0.01%`.
   > The diff coverage is `40.00%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/hudi/pull/2176/graphs/tree.svg?width=650&height=150&src=pr&token=VTTXabwbs2)](https://codecov.io/gh/apache/hudi/pull/2176?src=pr&el=tree)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #2176      +/-   ##
   ============================================
   + Coverage     53.53%   53.55%   +0.01%     
   - Complexity     2770     2771       +1     
   ============================================
     Files           348      348              
     Lines         16099    16100       +1     
     Branches       1642     1643       +1     
   ============================================
   + Hits           8619     8622       +3     
   + Misses         6785     6782       -3     
   - Partials        695      696       +1     
   ```
   
   | Flag | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | hudicli | `38.37% <ø> (ø)` | `0.00 <ø> (ø)` | |
   | hudiclient | `100.00% <ø> (ø)` | `0.00 <ø> (ø)` | |
   | hudicommon | `55.34% <40.00%> (+0.02%)` | `0.00 <0.00> (ø)` | |
   | hudihadoopmr | `32.94% <ø> (ø)` | `0.00 <ø> (ø)` | |
   | hudispark | `65.58% <ø> (ø)` | `0.00 <ø> (ø)` | |
   | huditimelineservice | `65.30% <ø> (ø)` | `0.00 <ø> (ø)` | |
   | hudiutilities | `70.09% <ø> (ø)` | `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/2176?src=pr&el=tree) | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | [...di/common/table/timeline/HoodieActiveTimeline.java](https://codecov.io/gh/apache/hudi/pull/2176/diff?src=pr&el=tree#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL3RhYmxlL3RpbWVsaW5lL0hvb2RpZUFjdGl2ZVRpbWVsaW5lLmphdmE=) | `73.76% <40.00%> (-0.37%)` | `41.00 <0.00> (ø)` | |
   | [...ache/hudi/common/fs/inline/InMemoryFileSystem.java](https://codecov.io/gh/apache/hudi/pull/2176/diff?src=pr&el=tree#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL2ZzL2lubGluZS9Jbk1lbW9yeUZpbGVTeXN0ZW0uamF2YQ==) | `89.65% <0.00%> (+10.34%)` | `16.00% <0.00%> (+1.00%)` | |
   


----------------------------------------------------------------
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] leesf commented on a change in pull request #2176: [HUDI-1327] Introduce base implementation of hudi-flink-client

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



##########
File path: hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/index/state/FlinkInMemoryStateIndex.java
##########
@@ -0,0 +1,135 @@
+/*
+ * 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.index.state;
+
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.client.common.HoodieEngineContext;
+import org.apache.hudi.client.common.HoodieFlinkEngineContext;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordLocation;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieIndexException;
+import org.apache.hudi.index.FlinkHoodieIndex;
+import org.apache.hudi.table.HoodieTable;
+
+import org.apache.flink.api.common.state.MapState;
+import org.apache.flink.api.common.state.MapStateDescriptor;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.util.List;
+
+/**
+ * Hoodie index implementation backed by flink state.
+ *
+ * @param <T> type of payload
+ */
+public class FlinkInMemoryStateIndex<T extends HoodieRecordPayload> extends FlinkHoodieIndex<T> {
+
+  private static final Logger LOG = LogManager.getLogger(FlinkInMemoryStateIndex.class);
+  private MapState<HoodieKey, HoodieRecordLocation> mapState;
+
+  public FlinkInMemoryStateIndex(HoodieFlinkEngineContext context, HoodieWriteConfig config) {
+    super(config);
+    if (context.getRuntimeContext() != null) {
+      MapStateDescriptor<HoodieKey, HoodieRecordLocation> indexStateDesc =
+          new MapStateDescriptor<>("indexState", TypeInformation.of(HoodieKey.class), TypeInformation.of(HoodieRecordLocation.class));
+      if (context.getRuntimeContext() != null) {
+        mapState = context.getRuntimeContext().getMapState(indexStateDesc);
+      }
+    }
+  }
+
+  @Override
+  public List<HoodieRecord<T>> tagLocation(List<HoodieRecord<T>> records,
+                                           HoodieEngineContext context,
+                                           HoodieTable<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> hoodieTable) throws HoodieIndexException {
+    return context.map(records, record -> {
+      try {
+        if (mapState.contains(record.getKey())) {
+          record.unseal();
+          record.setCurrentLocation(mapState.get(record.getKey()));
+          record.seal();
+        }
+      } catch (Exception e) {
+        LOG.error(String.format("Tag record location failed, key = %s, %s", record.getRecordKey(), e.getMessage()));
+      }
+      return record;
+    }, 0);
+  }
+
+  @Override
+  public List<WriteStatus> updateLocation(List<WriteStatus> writeStatuses,
+                                          HoodieEngineContext context,
+                                          HoodieTable<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> hoodieTable) throws HoodieIndexException {
+    return context.map(writeStatuses, writeStatus -> {
+      for (HoodieRecord record : writeStatus.getWrittenRecords()) {
+        if (!writeStatus.isErrored(record.getKey())) {
+          HoodieKey key = record.getKey();
+          Option<HoodieRecordLocation> newLocation = record.getNewLocation();
+          if (newLocation.isPresent()) {
+            try {
+              mapState.put(key, newLocation.get());
+            } catch (Exception e) {
+              LOG.error(String.format("Update record location failed, key = %s, %s", record.getRecordKey(), e.getMessage()));

Review comment:
       ditto




----------------------------------------------------------------
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 #2176: [HUDI-1327] Introduce base implementation of hudi-flink-client

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



##########
File path: hudi-flink-writer/src/main/java/org/apache/hudi/HudiFlinkStreamer.java
##########
@@ -0,0 +1,164 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi;
+
+import com.beust.jcommander.IStringConverter;
+import com.beust.jcommander.JCommander;
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.ParameterException;
+import org.apache.flink.api.common.serialization.SimpleStringSchema;
+import org.apache.flink.api.common.typeinfo.TypeHint;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.state.filesystem.FsStateBackend;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload;
+import org.apache.hudi.constant.Operation;
+import org.apache.hudi.operator.InstantGenerateOperator;
+import org.apache.hudi.operator.KeyedWriteProcessFunction;
+import org.apache.hudi.operator.WriteProcessOperator;
+import org.apache.hudi.sink.CommitSink;
+import org.apache.hudi.source.KafkaJson2HoodieRecord;
+import org.apache.hudi.util.StreamerUtil;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Properties;
+
+public class HudiFlinkStreamer {
+  public static void main(String[] args) throws Exception {
+    StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+
+    final Config cfg = new Config();
+    JCommander cmd = new JCommander(cfg, null, args);
+    if (cfg.help || args.length == 0) {
+      cmd.usage();
+      System.exit(1);
+    }
+    env.enableCheckpointing(cfg.checkpointInterval);
+    env.getConfig().setGlobalJobParameters(cfg);
+    env.getCheckpointConfig().setMaxConcurrentCheckpoints(1);

Review comment:
       done

##########
File path: hudi-flink-writer/src/main/java/org/apache/hudi/HudiFlinkStreamer.java
##########
@@ -0,0 +1,164 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi;
+
+import com.beust.jcommander.IStringConverter;
+import com.beust.jcommander.JCommander;
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.ParameterException;
+import org.apache.flink.api.common.serialization.SimpleStringSchema;
+import org.apache.flink.api.common.typeinfo.TypeHint;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.state.filesystem.FsStateBackend;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload;
+import org.apache.hudi.constant.Operation;
+import org.apache.hudi.operator.InstantGenerateOperator;
+import org.apache.hudi.operator.KeyedWriteProcessFunction;
+import org.apache.hudi.operator.WriteProcessOperator;
+import org.apache.hudi.sink.CommitSink;
+import org.apache.hudi.source.KafkaJson2HoodieRecord;
+import org.apache.hudi.util.StreamerUtil;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Properties;
+
+public class HudiFlinkStreamer {
+  public static void main(String[] args) throws Exception {
+    StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+
+    final Config cfg = new Config();
+    JCommander cmd = new JCommander(cfg, null, args);
+    if (cfg.help || args.length == 0) {
+      cmd.usage();
+      System.exit(1);
+    }
+    env.enableCheckpointing(cfg.checkpointInterval);
+    env.getConfig().setGlobalJobParameters(cfg);
+    env.getCheckpointConfig().setMaxConcurrentCheckpoints(1);
+    env.disableOperatorChaining();
+
+    if (cfg.flinkCheckPointPath != null) {
+      env.setStateBackend(new FsStateBackend(cfg.flinkCheckPointPath));
+    }
+
+    Properties kafkaProps = StreamerUtil.getKafkaProps(cfg);
+
+    // read from kafka source
+    DataStream<HoodieRecord> inputRecords =
+        env.addSource(new FlinkKafkaConsumer<>(cfg.kafkaTopic, new SimpleStringSchema(), kafkaProps))
+            .map(new KafkaJson2HoodieRecord(cfg))
+            .name("kafka_to_hudi_record")
+            .uid("kafka_to_hudi_record_uid");
+
+    inputRecords.transform(InstantGenerateOperator.NAME, TypeInformation.of(HoodieRecord.class), new InstantGenerateOperator())
+        .setParallelism(1)

Review comment:
       done




----------------------------------------------------------------
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 pull request #2176: [HUDI-1327] Introduce base implementation of hudi-flink-client

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


   Yes,will write one soon
   
   发自我的iPhone
   
   > 在 2020年11月19日,10:17,vinoyang <no...@github.com> 写道:
   > 
   > 
   > @wangxianghu Please consider writing an English/Chinese blog give some introduction about this feature.
   > 
   > —
   > You are receiving this because you were mentioned.
   > Reply to this email directly, view it on GitHub, or unsubscribe.
   


----------------------------------------------------------------
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 #2176: [HUDI-1327] Introduce base implementation of hudi-flink-client

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



##########
File path: hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/FlinkTaskContextSupplier.java
##########
@@ -0,0 +1,65 @@
+/*
+ * 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.client;
+
+import org.apache.hudi.client.common.EngineProperty;
+import org.apache.hudi.client.common.TaskContextSupplier;
+import org.apache.hudi.common.util.Option;
+
+import org.apache.flink.api.common.functions.RuntimeContext;
+
+import java.util.function.Supplier;
+
+/**
+ * Flink task context supplier.
+ */
+public class FlinkTaskContextSupplier extends TaskContextSupplier {
+  private org.apache.flink.api.common.functions.RuntimeContext flinkRuntimeContext;

Review comment:
       > would be changed to `RuntimeContext`?
   
   done




----------------------------------------------------------------
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 pull request #2176: [HUDI-1327] Introduce base implementation of hudi-flink-client

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


   > @wangxianghu big congrats! looking forward to the upcoming flink integration works
   
   Thanks, gary. I will improve and enhance it continuously 


----------------------------------------------------------------
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] leesf commented on a change in pull request #2176: [HUDI-1327] Introduce base implementation of hudi-flink-client

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



##########
File path: hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/FlinkTaskContextSupplier.java
##########
@@ -0,0 +1,65 @@
+/*
+ * 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.client;
+
+import org.apache.hudi.client.common.EngineProperty;
+import org.apache.hudi.client.common.TaskContextSupplier;
+import org.apache.hudi.common.util.Option;
+
+import org.apache.flink.api.common.functions.RuntimeContext;
+
+import java.util.function.Supplier;
+
+/**
+ * Flink task context supplier.
+ */
+public class FlinkTaskContextSupplier extends TaskContextSupplier {
+  private org.apache.flink.api.common.functions.RuntimeContext flinkRuntimeContext;

Review comment:
       would be changed to `RuntimeContext`?




----------------------------------------------------------------
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 #2176: [HUDI-1327] Introduce base implementation of hudi-flink-client

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


   @wangxianghu Please consider writing an English/Chinese blog give some introduction about this feature.


----------------------------------------------------------------
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 #2176: [HUDI-1327] Introduce base implementation of hudi-flink-client

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



##########
File path: hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/index/state/FlinkInMemoryStateIndex.java
##########
@@ -0,0 +1,135 @@
+/*
+ * 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.index.state;
+
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.client.common.HoodieEngineContext;
+import org.apache.hudi.client.common.HoodieFlinkEngineContext;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordLocation;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieIndexException;
+import org.apache.hudi.index.FlinkHoodieIndex;
+import org.apache.hudi.table.HoodieTable;
+
+import org.apache.flink.api.common.state.MapState;
+import org.apache.flink.api.common.state.MapStateDescriptor;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.util.List;
+
+/**
+ * Hoodie index implementation backed by flink state.
+ *
+ * @param <T> type of payload
+ */
+public class FlinkInMemoryStateIndex<T extends HoodieRecordPayload> extends FlinkHoodieIndex<T> {

Review comment:
       > is this always in memory? or can it be rocksDB backed as well?
   
   For now, if we configured  `--flink-checkpoint-path`,  the index data will be periodically persisted to HDFS.
   `RockDB` backend has some issues now, I'll fix 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 #2176: [HUDI-1327] Introduce base implementation of hudi-flink-client

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


   # [Codecov](https://codecov.io/gh/apache/hudi/pull/2176?src=pr&el=h1) Report
   > Merging [#2176](https://codecov.io/gh/apache/hudi/pull/2176?src=pr&el=desc) (07097e7) into [master](https://codecov.io/gh/apache/hudi/commit/0364498ae3be553f7c9fab78469283a6958da2ff?el=desc) (0364498) will **increase** coverage by `0.01%`.
   > The diff coverage is `40.00%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/hudi/pull/2176/graphs/tree.svg?width=650&height=150&src=pr&token=VTTXabwbs2)](https://codecov.io/gh/apache/hudi/pull/2176?src=pr&el=tree)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #2176      +/-   ##
   ============================================
   + Coverage     53.53%   53.55%   +0.01%     
   - Complexity     2770     2771       +1     
   ============================================
     Files           348      348              
     Lines         16099    16100       +1     
     Branches       1642     1643       +1     
   ============================================
   + Hits           8619     8622       +3     
   + Misses         6785     6782       -3     
   - Partials        695      696       +1     
   ```
   
   | Flag | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | hudicli | `38.37% <ø> (ø)` | `0.00 <ø> (ø)` | |
   | hudiclient | `100.00% <ø> (ø)` | `0.00 <ø> (ø)` | |
   | hudicommon | `55.34% <40.00%> (+0.02%)` | `0.00 <0.00> (ø)` | |
   | hudihadoopmr | `32.94% <ø> (ø)` | `0.00 <ø> (ø)` | |
   | hudispark | `65.58% <ø> (ø)` | `0.00 <ø> (ø)` | |
   | huditimelineservice | `65.30% <ø> (ø)` | `0.00 <ø> (ø)` | |
   | hudiutilities | `70.09% <ø> (ø)` | `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/2176?src=pr&el=tree) | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | [...di/common/table/timeline/HoodieActiveTimeline.java](https://codecov.io/gh/apache/hudi/pull/2176/diff?src=pr&el=tree#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL3RhYmxlL3RpbWVsaW5lL0hvb2RpZUFjdGl2ZVRpbWVsaW5lLmphdmE=) | `73.76% <40.00%> (-0.37%)` | `41.00 <0.00> (ø)` | |
   | [...ache/hudi/common/fs/inline/InMemoryFileSystem.java](https://codecov.io/gh/apache/hudi/pull/2176/diff?src=pr&el=tree#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL2ZzL2lubGluZS9Jbk1lbW9yeUZpbGVTeXN0ZW0uamF2YQ==) | `89.65% <0.00%> (+10.34%)` | `16.00% <0.00%> (+1.00%)` | |
   


----------------------------------------------------------------
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 pull request #2176: [HUDI-1327] Introduce base implementation of hudi-flink-client

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


   Hi, @yanghua @leesf @garyli1019 
   any other concern 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 #2176: [HUDI-1327] Introduce base implementation of hudi-flink-client

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



##########
File path: hudi-flink-writer/src/main/java/org/apache/hudi/HudiFlinkStreamer.java
##########
@@ -0,0 +1,164 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi;
+
+import com.beust.jcommander.IStringConverter;
+import com.beust.jcommander.JCommander;
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.ParameterException;
+import org.apache.flink.api.common.serialization.SimpleStringSchema;
+import org.apache.flink.api.common.typeinfo.TypeHint;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.state.filesystem.FsStateBackend;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload;
+import org.apache.hudi.constant.Operation;
+import org.apache.hudi.operator.InstantGenerateOperator;
+import org.apache.hudi.operator.KeyedWriteProcessFunction;
+import org.apache.hudi.operator.WriteProcessOperator;
+import org.apache.hudi.sink.CommitSink;
+import org.apache.hudi.source.KafkaJson2HoodieRecord;
+import org.apache.hudi.util.StreamerUtil;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Properties;
+
+public class HudiFlinkStreamer {
+  public static void main(String[] args) throws Exception {
+    StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+
+    final Config cfg = new Config();
+    JCommander cmd = new JCommander(cfg, null, args);
+    if (cfg.help || args.length == 0) {
+      cmd.usage();
+      System.exit(1);
+    }
+    env.enableCheckpointing(cfg.checkpointInterval);
+    env.getConfig().setGlobalJobParameters(cfg);
+    env.getCheckpointConfig().setMaxConcurrentCheckpoints(1);
+    env.disableOperatorChaining();
+
+    if (cfg.flinkCheckPointPath != null) {
+      env.setStateBackend(new FsStateBackend(cfg.flinkCheckPointPath));
+    }
+
+    Properties kafkaProps = StreamerUtil.getKafkaProps(cfg);
+
+    // read from kafka source
+    DataStream<HoodieRecord> inputRecords =
+        env.addSource(new FlinkKafkaConsumer<>(cfg.kafkaTopic, new SimpleStringSchema(), kafkaProps))
+            .map(new KafkaJson2HoodieRecord(cfg))
+            .name("kafka_to_hudi_record")
+            .uid("kafka_to_hudi_record_uid");
+
+    inputRecords.transform(InstantGenerateOperator.NAME, TypeInformation.of(HoodieRecord.class), new InstantGenerateOperator())
+        .setParallelism(1)
+        .keyBy(HoodieRecord::getPartitionPath)
+        .transform(WriteProcessOperator.NAME, TypeInformation.of(new TypeHint<Tuple3<String, List<WriteStatus>, Integer>>() {
+        }), new WriteProcessOperator(new KeyedWriteProcessFunction())).name("write_process").uid("write_process_uid")

Review comment:
       The indent is bad. 

##########
File path: hudi-flink-writer/src/main/java/org/apache/hudi/HudiFlinkStreamer.java
##########
@@ -0,0 +1,164 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi;
+
+import com.beust.jcommander.IStringConverter;
+import com.beust.jcommander.JCommander;
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.ParameterException;
+import org.apache.flink.api.common.serialization.SimpleStringSchema;
+import org.apache.flink.api.common.typeinfo.TypeHint;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.state.filesystem.FsStateBackend;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload;
+import org.apache.hudi.constant.Operation;
+import org.apache.hudi.operator.InstantGenerateOperator;
+import org.apache.hudi.operator.KeyedWriteProcessFunction;
+import org.apache.hudi.operator.WriteProcessOperator;
+import org.apache.hudi.sink.CommitSink;
+import org.apache.hudi.source.KafkaJson2HoodieRecord;
+import org.apache.hudi.util.StreamerUtil;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Properties;
+
+public class HudiFlinkStreamer {
+  public static void main(String[] args) throws Exception {
+    StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+
+    final Config cfg = new Config();
+    JCommander cmd = new JCommander(cfg, null, args);
+    if (cfg.help || args.length == 0) {
+      cmd.usage();
+      System.exit(1);
+    }
+    env.enableCheckpointing(cfg.checkpointInterval);
+    env.getConfig().setGlobalJobParameters(cfg);
+    env.getCheckpointConfig().setMaxConcurrentCheckpoints(1);
+    env.disableOperatorChaining();
+
+    if (cfg.flinkCheckPointPath != null) {
+      env.setStateBackend(new FsStateBackend(cfg.flinkCheckPointPath));
+    }
+
+    Properties kafkaProps = StreamerUtil.getKafkaProps(cfg);
+
+    // read from kafka source
+    DataStream<HoodieRecord> inputRecords =
+        env.addSource(new FlinkKafkaConsumer<>(cfg.kafkaTopic, new SimpleStringSchema(), kafkaProps))
+            .map(new KafkaJson2HoodieRecord(cfg))
+            .name("kafka_to_hudi_record")
+            .uid("kafka_to_hudi_record_uid");
+
+    inputRecords.transform(InstantGenerateOperator.NAME, TypeInformation.of(HoodieRecord.class), new InstantGenerateOperator())
+        .setParallelism(1)
+        .keyBy(HoodieRecord::getPartitionPath)
+        .transform(WriteProcessOperator.NAME, TypeInformation.of(new TypeHint<Tuple3<String, List<WriteStatus>, Integer>>() {
+        }), new WriteProcessOperator(new KeyedWriteProcessFunction())).name("write_process").uid("write_process_uid")
+        .setParallelism(env.getParallelism())
+        .addSink(new CommitSink()).name("commit_sink").uid("commit_sink_uid")
+        .setParallelism(1);
+
+    env.execute("Hudi write via Flink");

Review comment:
       `Hudi write job via Flink`? And can we add some variables to distinguish different hudi job? e.g. table name?

##########
File path: hudi-flink-writer/src/main/java/org/apache/hudi/operator/InstantGenerateOperator.java
##########
@@ -0,0 +1,208 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.operator;
+
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hudi.HudiFlinkStreamer;
+import org.apache.hudi.client.FlinkTaskContextSupplier;
+import org.apache.hudi.client.HoodieFlinkWriteClient;
+import org.apache.hudi.client.common.HoodieFlinkEngineContext;
+import org.apache.hudi.client.common.TaskContextSupplier;
+import org.apache.hudi.common.config.SerializableConfiguration;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieTableType;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.timeline.HoodieTimeline;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.util.StreamerUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+
+public class InstantGenerateOperator extends AbstractStreamOperator<HoodieRecord> implements OneInputStreamOperator<HoodieRecord, HoodieRecord> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(InstantGenerateOperator.class);
+  public static final String NAME = "InstantGenerateOperator";
+  private HudiFlinkStreamer.Config cfg;
+  private HoodieWriteConfig writeConfig;
+  private HoodieFlinkWriteClient writeClient;
+  private SerializableConfiguration serializableHadoopConf;
+  private transient FileSystem fs;
+  private String latestInstant = "";
+  List<String> latestInstantList = new ArrayList<>(1);
+  private transient ListState<String> latestInstantState;
+  private List<StreamRecord> records = new LinkedList();

Review comment:
       It seems `bufferedRecords` sounds better?

##########
File path: hudi-flink-writer/src/main/java/org/apache/hudi/HudiFlinkStreamer.java
##########
@@ -0,0 +1,164 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi;
+
+import com.beust.jcommander.IStringConverter;
+import com.beust.jcommander.JCommander;
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.ParameterException;
+import org.apache.flink.api.common.serialization.SimpleStringSchema;
+import org.apache.flink.api.common.typeinfo.TypeHint;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.state.filesystem.FsStateBackend;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload;
+import org.apache.hudi.constant.Operation;
+import org.apache.hudi.operator.InstantGenerateOperator;
+import org.apache.hudi.operator.KeyedWriteProcessFunction;
+import org.apache.hudi.operator.WriteProcessOperator;
+import org.apache.hudi.sink.CommitSink;
+import org.apache.hudi.source.KafkaJson2HoodieRecord;
+import org.apache.hudi.util.StreamerUtil;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Properties;
+
+public class HudiFlinkStreamer {
+  public static void main(String[] args) throws Exception {
+    StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+
+    final Config cfg = new Config();
+    JCommander cmd = new JCommander(cfg, null, args);
+    if (cfg.help || args.length == 0) {
+      cmd.usage();
+      System.exit(1);
+    }
+    env.enableCheckpointing(cfg.checkpointInterval);
+    env.getConfig().setGlobalJobParameters(cfg);
+    env.getCheckpointConfig().setMaxConcurrentCheckpoints(1);
+    env.disableOperatorChaining();
+
+    if (cfg.flinkCheckPointPath != null) {
+      env.setStateBackend(new FsStateBackend(cfg.flinkCheckPointPath));
+    }
+
+    Properties kafkaProps = StreamerUtil.getKafkaProps(cfg);
+
+    // read from kafka source
+    DataStream<HoodieRecord> inputRecords =
+        env.addSource(new FlinkKafkaConsumer<>(cfg.kafkaTopic, new SimpleStringSchema(), kafkaProps))
+            .map(new KafkaJson2HoodieRecord(cfg))
+            .name("kafka_to_hudi_record")
+            .uid("kafka_to_hudi_record_uid");
+
+    inputRecords.transform(InstantGenerateOperator.NAME, TypeInformation.of(HoodieRecord.class), new InstantGenerateOperator())
+        .setParallelism(1)

Review comment:
       Add a description about why hard code this parallelism.

##########
File path: hudi-flink-writer/src/main/java/org/apache/hudi/operator/InstantGenerateOperator.java
##########
@@ -0,0 +1,208 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.operator;
+
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hudi.HudiFlinkStreamer;
+import org.apache.hudi.client.FlinkTaskContextSupplier;
+import org.apache.hudi.client.HoodieFlinkWriteClient;
+import org.apache.hudi.client.common.HoodieFlinkEngineContext;
+import org.apache.hudi.client.common.TaskContextSupplier;
+import org.apache.hudi.common.config.SerializableConfiguration;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieTableType;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.timeline.HoodieTimeline;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.util.StreamerUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+
+public class InstantGenerateOperator extends AbstractStreamOperator<HoodieRecord> implements OneInputStreamOperator<HoodieRecord, HoodieRecord> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(InstantGenerateOperator.class);
+  public static final String NAME = "InstantGenerateOperator";
+  private HudiFlinkStreamer.Config cfg;
+  private HoodieWriteConfig writeConfig;
+  private HoodieFlinkWriteClient writeClient;
+  private SerializableConfiguration serializableHadoopConf;
+  private transient FileSystem fs;
+  private String latestInstant = "";
+  List<String> latestInstantList = new ArrayList<>(1);

Review comment:
       keep the same style as others?  add the access modifier

##########
File path: hudi-flink-writer/src/main/java/org/apache/hudi/operator/InstantGenerateOperator.java
##########
@@ -0,0 +1,208 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.operator;
+
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hudi.HudiFlinkStreamer;
+import org.apache.hudi.client.FlinkTaskContextSupplier;
+import org.apache.hudi.client.HoodieFlinkWriteClient;
+import org.apache.hudi.client.common.HoodieFlinkEngineContext;
+import org.apache.hudi.client.common.TaskContextSupplier;
+import org.apache.hudi.common.config.SerializableConfiguration;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieTableType;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.timeline.HoodieTimeline;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.util.StreamerUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+
+public class InstantGenerateOperator extends AbstractStreamOperator<HoodieRecord> implements OneInputStreamOperator<HoodieRecord, HoodieRecord> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(InstantGenerateOperator.class);
+  public static final String NAME = "InstantGenerateOperator";

Review comment:
       split the static fields and non-static fields

##########
File path: hudi-flink-writer/src/main/java/org/apache/hudi/operator/InstantGenerateOperator.java
##########
@@ -0,0 +1,208 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.operator;
+
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hudi.HudiFlinkStreamer;
+import org.apache.hudi.client.FlinkTaskContextSupplier;
+import org.apache.hudi.client.HoodieFlinkWriteClient;
+import org.apache.hudi.client.common.HoodieFlinkEngineContext;
+import org.apache.hudi.client.common.TaskContextSupplier;
+import org.apache.hudi.common.config.SerializableConfiguration;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieTableType;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.timeline.HoodieTimeline;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.util.StreamerUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+
+public class InstantGenerateOperator extends AbstractStreamOperator<HoodieRecord> implements OneInputStreamOperator<HoodieRecord, HoodieRecord> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(InstantGenerateOperator.class);
+  public static final String NAME = "InstantGenerateOperator";
+  private HudiFlinkStreamer.Config cfg;
+  private HoodieWriteConfig writeConfig;
+  private HoodieFlinkWriteClient writeClient;
+  private SerializableConfiguration serializableHadoopConf;
+  private transient FileSystem fs;
+  private String latestInstant = "";
+  List<String> latestInstantList = new ArrayList<>(1);
+  private transient ListState<String> latestInstantState;
+  private List<StreamRecord> records = new LinkedList();
+  private transient ListState<StreamRecord> recordsState;
+  private Integer commitTimeout;
+
+  @Override
+  public void processElement(StreamRecord<HoodieRecord> streamRecord) throws Exception {
+    if (streamRecord.getValue() != null) {
+      records.add(streamRecord);
+      output.collect(streamRecord);
+    }
+  }
+
+  @Override
+  public void open() throws Exception {
+    super.open();
+    // get configs from runtimeContext
+    cfg = (HudiFlinkStreamer.Config) getRuntimeContext().getExecutionConfig().getGlobalJobParameters();
+
+    // timeout
+    commitTimeout = Integer.valueOf(cfg.flinkCommitTimeout);
+
+    // hadoopConf
+    serializableHadoopConf = new SerializableConfiguration(StreamerUtil.getHadoopConf());
+
+    // Hadoop FileSystem
+    fs = FSUtils.getFs(cfg.targetBasePath, serializableHadoopConf.get());
+
+    // HoodieWriteConfig
+    writeConfig = StreamerUtil.getHoodieClientConfig(cfg);
+
+    TaskContextSupplier taskContextSupplier = new FlinkTaskContextSupplier(null);
+
+    // writeClient
+    writeClient = new HoodieFlinkWriteClient(new HoodieFlinkEngineContext(taskContextSupplier), writeConfig);
+
+    // init table, create it if not exists.
+    initTable();
+  }
+
+  @Override
+  public void prepareSnapshotPreBarrier(long checkpointId) throws Exception {
+    super.prepareSnapshotPreBarrier(checkpointId);
+    // check whether the last instant is completed, if not, wait 10s and then throws an exception
+    if (!StringUtils.isNullOrEmpty(latestInstant)) {
+      doChecker();
+      // last instant completed, set it empty
+      latestInstant = "";
+    }
+
+    // no data no new instant
+    if (!records.isEmpty()) {
+      latestInstant = startNewInstant(checkpointId);
+    }
+    super.prepareSnapshotPreBarrier(checkpointId);

Review comment:
       Why do two times call this method?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] garyli1019 commented on a change in pull request #2176: [HUDI-1327] Introduce base implementation of hudi-flink-client

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



##########
File path: hudi-flink-writer/src/main/java/org/apache/hudi/schema/FilebasedSchemaProvider.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.schema;
+
+import org.apache.avro.Schema;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.exception.HoodieIOException;
+import org.apache.hudi.util.StreamerUtil;
+
+import java.io.IOException;
+import java.util.Collections;
+
+/**
+ * A simple schema provider, that reads off files on DFS.
+ */
+public class FilebasedSchemaProvider extends SchemaProvider {

Review comment:
       ah ok, I got confused by the class name. Should we move this to `TableSchemaResolver` then, so we have schema related stuff in one place. WDYT?




----------------------------------------------------------------
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 #2176: [HUDI-1327] Introduce base implementation of hudi-flink-client

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


   > This is a huge step! worth tweeting about (if none does, I ll do it :) )
   
   OK, Will tweet it soon.


----------------------------------------------------------------
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 #2176: [HUDI-1327] Introduce base implementation of hudi-flink-client

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



##########
File path: hudi-flink-writer/src/main/java/org/apache/hudi/HudiFlinkStreamer.java
##########
@@ -0,0 +1,188 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi;
+
+import com.beust.jcommander.IStringConverter;
+import com.beust.jcommander.JCommander;
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.ParameterException;
+import org.apache.flink.api.common.serialization.SimpleStringSchema;
+import org.apache.flink.api.common.typeinfo.TypeHint;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.state.filesystem.FsStateBackend;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload;
+import org.apache.hudi.constant.Operation;
+import org.apache.hudi.operator.InstantGenerateOperator;
+import org.apache.hudi.operator.KeyedWriteProcessFunction;
+import org.apache.hudi.operator.KeyedWriteProcessOperator;
+import org.apache.hudi.sink.CommitSink;
+import org.apache.hudi.source.JsonStringToHoodieRecordMapFunction;
+import org.apache.hudi.util.StreamerUtil;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Objects;
+import java.util.Properties;
+
+/**
+ * An Utility which can incrementally consume data from Kafka and apply it to the target table.
+ * currently, it only support MOR table and insert, upsert operation.

Review comment:
       > can we make sure the program throws errors if using COW etc.
   
   I'll give a try




----------------------------------------------------------------
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 #2176: [HUDI-1327] Introduce base implementation of hudi-flink-client

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



##########
File path: hudi-flink-writer/src/main/java/org/apache/hudi/source/KafkaJson2HoodieRecord.java
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.source;
+
+import org.apache.avro.generic.GenericRecord;
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.hudi.HudiFlinkStreamer;
+import org.apache.hudi.avro.HoodieAvroUtils;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.keygen.KeyGenerator;
+import org.apache.hudi.schema.FilebasedSchemaProvider;
+import org.apache.hudi.util.AvroConvertor;
+import org.apache.hudi.util.StreamerUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+public class KafkaJson2HoodieRecord implements MapFunction<String, HoodieRecord> {
+
+  private static Logger LOG = LoggerFactory.getLogger(KafkaJson2HoodieRecord.class);
+
+  private final HudiFlinkStreamer.Config cfg;
+  private TypedProperties props;
+  private KeyGenerator keyGenerator;
+  private AvroConvertor convertor;

Review comment:
       Add a new empty line.

##########
File path: hudi-flink-writer/src/main/java/org/apache/hudi/operator/KeyedWriteProcessFunction.java
##########
@@ -0,0 +1,164 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.operator;
+
+import org.apache.hudi.HudiFlinkStreamer;
+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.HoodieTableType;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.common.table.timeline.HoodieTimeline;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieFlinkStreamerException;
+import org.apache.hudi.util.StreamerUtil;
+
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.configuration.Configuration;
+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.util.Collector;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.LinkedList;
+import java.util.List;
+
+public class KeyedWriteProcessFunction extends KeyedProcessFunction<String, HoodieRecord, Tuple3<String, List<WriteStatus>, Integer>> implements CheckpointedFunction {
+
+  private static final Logger LOG = LoggerFactory.getLogger(KeyedWriteProcessFunction.class);
+  private List<HoodieRecord> records = new LinkedList<>();
+  private Collector<Tuple3<String, List<WriteStatus>, Integer>> output;
+  private int indexOfThisSubtask;
+  private String latestInstant;
+  private boolean hasRecordsIn;
+
+  /**
+   * Job conf.
+   */
+  private HudiFlinkStreamer.Config cfg;
+  /**
+   * Serializable hadoop conf.
+   */
+  private SerializableConfiguration serializableHadoopConf;
+  /**
+   * HoodieWriteConfig.
+   */
+  private HoodieWriteConfig writeConfig;

Review comment:
       Can be a local variable.

##########
File path: hudi-flink-writer/src/main/java/org/apache/hudi/source/KafkaJson2HoodieRecord.java
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.source;
+
+import org.apache.avro.generic.GenericRecord;
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.hudi.HudiFlinkStreamer;
+import org.apache.hudi.avro.HoodieAvroUtils;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.keygen.KeyGenerator;
+import org.apache.hudi.schema.FilebasedSchemaProvider;
+import org.apache.hudi.util.AvroConvertor;
+import org.apache.hudi.util.StreamerUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+public class KafkaJson2HoodieRecord implements MapFunction<String, HoodieRecord> {

Review comment:
       It would be better to add `xxxFunction` as the suffix. wdyt?

##########
File path: hudi-flink-writer/src/main/java/org/apache/hudi/operator/KeyedWriteProcessFunction.java
##########
@@ -0,0 +1,164 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.operator;
+
+import org.apache.hudi.HudiFlinkStreamer;
+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.HoodieTableType;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.common.table.timeline.HoodieTimeline;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieFlinkStreamerException;
+import org.apache.hudi.util.StreamerUtil;
+
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.configuration.Configuration;
+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.util.Collector;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.LinkedList;
+import java.util.List;
+
+public class KeyedWriteProcessFunction extends KeyedProcessFunction<String, HoodieRecord, Tuple3<String, List<WriteStatus>, Integer>> implements CheckpointedFunction {
+
+  private static final Logger LOG = LoggerFactory.getLogger(KeyedWriteProcessFunction.class);
+  private List<HoodieRecord> records = new LinkedList<>();
+  private Collector<Tuple3<String, List<WriteStatus>, Integer>> output;
+  private int indexOfThisSubtask;
+  private String latestInstant;
+  private boolean hasRecordsIn;
+
+  /**
+   * Job conf.
+   */
+  private HudiFlinkStreamer.Config cfg;
+  /**
+   * Serializable hadoop conf.
+   */
+  private SerializableConfiguration serializableHadoopConf;
+  /**
+   * HoodieWriteConfig.
+   */
+  private HoodieWriteConfig writeConfig;
+
+  /**
+   * Write Client.
+   */
+  private transient HoodieFlinkWriteClient writeClient;
+
+  @Override
+  public void open(Configuration parameters) throws Exception {
+    super.open(parameters);
+
+    indexOfThisSubtask = getRuntimeContext().getIndexOfThisSubtask();
+
+    cfg = (HudiFlinkStreamer.Config) getRuntimeContext().getExecutionConfig().getGlobalJobParameters();
+
+    // hadoopConf
+    serializableHadoopConf = new SerializableConfiguration(new org.apache.hadoop.conf.Configuration());
+    // HoodieWriteConfig
+    writeConfig = StreamerUtil.getHoodieClientConfig(cfg);
+
+    HoodieFlinkEngineContext context = new HoodieFlinkEngineContext(serializableHadoopConf, new FlinkTaskContextSupplier(getRuntimeContext()));
+
+    // writeClient
+    writeClient = new HoodieFlinkWriteClient<>(context, writeConfig);
+  }
+
+  @Override
+  public void snapshotState(FunctionSnapshotContext context) throws Exception {
+    // get latest requested instant
+    String commitType = cfg.tableType.equals(HoodieTableType.COPY_ON_WRITE.name()) ? HoodieTimeline.COMMIT_ACTION : HoodieTimeline.DELTA_COMMIT_ACTION;
+    List<String> latestInstants = writeClient.getInflightsAndRequestedInstants(commitType);
+    latestInstant = latestInstants.isEmpty() ? null : latestInstants.get(0);
+
+    if (output != null && latestInstant != null && records.size() > 0) {
+      hasRecordsIn = true;
+      String instantTimestamp = latestInstant;
+      LOG.info("Upsert records, subtask id = [{}]  checkpoint_id = [{}}] instant = [{}], record size = [{}]", indexOfThisSubtask, context.getCheckpointId(), instantTimestamp, records.size());
+
+      List<WriteStatus> writeStatus;
+      switch (cfg.operation) {
+        case INSERT:
+          writeStatus = writeClient.insert(records, instantTimestamp);
+          break;
+        case UPSERT:
+          writeStatus = writeClient.upsert(records, instantTimestamp);
+          break;
+        default:
+          throw new HoodieFlinkStreamerException("Unknown operation : " + cfg.operation);
+      }
+      output.collect(new Tuple3<>(instantTimestamp, writeStatus, indexOfThisSubtask));
+      records.clear();
+    }
+  }
+
+  @Override
+  public void initializeState(FunctionInitializationContext functionInitializationContext) throws Exception {
+    // no operation
+  }
+
+  @Override
+  public void processElement(HoodieRecord hoodieRecord, Context context, Collector<Tuple3<String, List<WriteStatus>, Integer>> collector) throws Exception {
+    records.add(hoodieRecord);

Review comment:
       I am thinking one thing: if the interval of the checkpoint is too long. If this buffer would cause OOM?

##########
File path: hudi-flink-writer/src/main/java/org/apache/hudi/operator/KeyedWriteProcessFunction.java
##########
@@ -0,0 +1,164 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.operator;
+
+import org.apache.hudi.HudiFlinkStreamer;
+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.HoodieTableType;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.common.table.timeline.HoodieTimeline;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieFlinkStreamerException;
+import org.apache.hudi.util.StreamerUtil;
+
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.configuration.Configuration;
+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.util.Collector;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.LinkedList;
+import java.util.List;
+
+public class KeyedWriteProcessFunction extends KeyedProcessFunction<String, HoodieRecord, Tuple3<String, List<WriteStatus>, Integer>> implements CheckpointedFunction {
+
+  private static final Logger LOG = LoggerFactory.getLogger(KeyedWriteProcessFunction.class);
+  private List<HoodieRecord> records = new LinkedList<>();
+  private Collector<Tuple3<String, List<WriteStatus>, Integer>> output;
+  private int indexOfThisSubtask;
+  private String latestInstant;
+  private boolean hasRecordsIn;
+
+  /**
+   * Job conf.
+   */
+  private HudiFlinkStreamer.Config cfg;
+  /**
+   * Serializable hadoop conf.
+   */
+  private SerializableConfiguration serializableHadoopConf;
+  /**
+   * HoodieWriteConfig.
+   */
+  private HoodieWriteConfig writeConfig;
+
+  /**
+   * Write Client.
+   */
+  private transient HoodieFlinkWriteClient writeClient;
+
+  @Override
+  public void open(Configuration parameters) throws Exception {
+    super.open(parameters);
+
+    indexOfThisSubtask = getRuntimeContext().getIndexOfThisSubtask();
+
+    cfg = (HudiFlinkStreamer.Config) getRuntimeContext().getExecutionConfig().getGlobalJobParameters();
+
+    // hadoopConf
+    serializableHadoopConf = new SerializableConfiguration(new org.apache.hadoop.conf.Configuration());
+    // HoodieWriteConfig
+    writeConfig = StreamerUtil.getHoodieClientConfig(cfg);
+
+    HoodieFlinkEngineContext context = new HoodieFlinkEngineContext(serializableHadoopConf, new FlinkTaskContextSupplier(getRuntimeContext()));
+
+    // writeClient
+    writeClient = new HoodieFlinkWriteClient<>(context, writeConfig);
+  }
+
+  @Override
+  public void snapshotState(FunctionSnapshotContext context) throws Exception {
+    // get latest requested instant
+    String commitType = cfg.tableType.equals(HoodieTableType.COPY_ON_WRITE.name()) ? HoodieTimeline.COMMIT_ACTION : HoodieTimeline.DELTA_COMMIT_ACTION;
+    List<String> latestInstants = writeClient.getInflightsAndRequestedInstants(commitType);
+    latestInstant = latestInstants.isEmpty() ? null : latestInstants.get(0);
+
+    if (output != null && latestInstant != null && records.size() > 0) {
+      hasRecordsIn = true;
+      String instantTimestamp = latestInstant;
+      LOG.info("Upsert records, subtask id = [{}]  checkpoint_id = [{}}] instant = [{}], record size = [{}]", indexOfThisSubtask, context.getCheckpointId(), instantTimestamp, records.size());
+
+      List<WriteStatus> writeStatus;
+      switch (cfg.operation) {
+        case INSERT:
+          writeStatus = writeClient.insert(records, instantTimestamp);
+          break;
+        case UPSERT:
+          writeStatus = writeClient.upsert(records, instantTimestamp);
+          break;
+        default:
+          throw new HoodieFlinkStreamerException("Unknown operation : " + cfg.operation);
+      }
+      output.collect(new Tuple3<>(instantTimestamp, writeStatus, indexOfThisSubtask));
+      records.clear();
+    }
+  }
+
+  @Override
+  public void initializeState(FunctionInitializationContext functionInitializationContext) throws Exception {
+    // no operation
+  }
+
+  @Override
+  public void processElement(HoodieRecord hoodieRecord, Context context, Collector<Tuple3<String, List<WriteStatus>, Integer>> collector) throws Exception {

Review comment:
       never used Exception.

##########
File path: hudi-flink-writer/src/main/java/org/apache/hudi/operator/KeyedWriteProcessFunction.java
##########
@@ -0,0 +1,164 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.operator;
+
+import org.apache.hudi.HudiFlinkStreamer;
+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.HoodieTableType;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.common.table.timeline.HoodieTimeline;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieFlinkStreamerException;
+import org.apache.hudi.util.StreamerUtil;
+
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.configuration.Configuration;
+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.util.Collector;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.LinkedList;
+import java.util.List;
+
+public class KeyedWriteProcessFunction extends KeyedProcessFunction<String, HoodieRecord, Tuple3<String, List<WriteStatus>, Integer>> implements CheckpointedFunction {
+
+  private static final Logger LOG = LoggerFactory.getLogger(KeyedWriteProcessFunction.class);
+  private List<HoodieRecord> records = new LinkedList<>();
+  private Collector<Tuple3<String, List<WriteStatus>, Integer>> output;
+  private int indexOfThisSubtask;
+  private String latestInstant;
+  private boolean hasRecordsIn;
+
+  /**
+   * Job conf.
+   */
+  private HudiFlinkStreamer.Config cfg;
+  /**
+   * Serializable hadoop conf.
+   */
+  private SerializableConfiguration serializableHadoopConf;
+  /**
+   * HoodieWriteConfig.
+   */
+  private HoodieWriteConfig writeConfig;
+
+  /**
+   * Write Client.
+   */
+  private transient HoodieFlinkWriteClient writeClient;
+
+  @Override
+  public void open(Configuration parameters) throws Exception {
+    super.open(parameters);
+
+    indexOfThisSubtask = getRuntimeContext().getIndexOfThisSubtask();
+
+    cfg = (HudiFlinkStreamer.Config) getRuntimeContext().getExecutionConfig().getGlobalJobParameters();
+
+    // hadoopConf
+    serializableHadoopConf = new SerializableConfiguration(new org.apache.hadoop.conf.Configuration());
+    // HoodieWriteConfig
+    writeConfig = StreamerUtil.getHoodieClientConfig(cfg);
+
+    HoodieFlinkEngineContext context = new HoodieFlinkEngineContext(serializableHadoopConf, new FlinkTaskContextSupplier(getRuntimeContext()));
+
+    // writeClient
+    writeClient = new HoodieFlinkWriteClient<>(context, writeConfig);
+  }
+
+  @Override
+  public void snapshotState(FunctionSnapshotContext context) throws Exception {
+    // get latest requested instant
+    String commitType = cfg.tableType.equals(HoodieTableType.COPY_ON_WRITE.name()) ? HoodieTimeline.COMMIT_ACTION : HoodieTimeline.DELTA_COMMIT_ACTION;
+    List<String> latestInstants = writeClient.getInflightsAndRequestedInstants(commitType);
+    latestInstant = latestInstants.isEmpty() ? null : latestInstants.get(0);
+
+    if (output != null && latestInstant != null && records.size() > 0) {
+      hasRecordsIn = true;
+      String instantTimestamp = latestInstant;
+      LOG.info("Upsert records, subtask id = [{}]  checkpoint_id = [{}}] instant = [{}], record size = [{}]", indexOfThisSubtask, context.getCheckpointId(), instantTimestamp, records.size());
+
+      List<WriteStatus> writeStatus;
+      switch (cfg.operation) {
+        case INSERT:
+          writeStatus = writeClient.insert(records, instantTimestamp);
+          break;
+        case UPSERT:
+          writeStatus = writeClient.upsert(records, instantTimestamp);
+          break;
+        default:
+          throw new HoodieFlinkStreamerException("Unknown operation : " + cfg.operation);
+      }
+      output.collect(new Tuple3<>(instantTimestamp, writeStatus, indexOfThisSubtask));
+      records.clear();
+    }
+  }
+
+  @Override
+  public void initializeState(FunctionInitializationContext functionInitializationContext) throws Exception {

Review comment:
       never used exception

##########
File path: hudi-flink-writer/src/main/java/org/apache/hudi/operator/WriteProcessOperator.java
##########
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.operator;
+
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.streaming.api.functions.KeyedProcessFunction;
+import org.apache.flink.streaming.api.operators.KeyedProcessOperator;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.List;
+
+public class WriteProcessOperator extends KeyedProcessOperator<String, HoodieRecord, Tuple3<String, List<WriteStatus>, Integer>> {
+  public static final String NAME = "WriteProcessOperator";
+  private static final Logger LOG = LoggerFactory.getLogger(WriteProcessOperator.class);
+  private KeyedWriteProcessFunction function;
+
+  public WriteProcessOperator(KeyedProcessFunction<String, HoodieRecord, Tuple3<String, List<WriteStatus>, Integer>> function) {
+    super(function);
+    this.function = (KeyedWriteProcessFunction) function;
+  }
+
+  @Override
+  public void snapshotState(StateSnapshotContext context) throws Exception {

Review comment:
       It would be better to explain why do we need to implement the `snapshotState` method both in UDF and operator?

##########
File path: hudi-flink-writer/src/main/java/org/apache/hudi/sink/CommitSink.java
##########
@@ -0,0 +1,150 @@
+/*
+ * 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.sink;
+
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
+import org.apache.hudi.HudiFlinkStreamer;
+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.util.Option;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.util.StreamerUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+public class CommitSink extends RichSinkFunction<Tuple3<String, List<WriteStatus>, Integer>> {
+  private static final Logger LOG = LoggerFactory.getLogger(CommitSink.class);
+  /**
+   * Job conf.
+   */
+  private HudiFlinkStreamer.Config cfg;
+
+  /**
+   * HoodieWriteConfig.
+   */
+  private HoodieWriteConfig writeConfig;
+
+  /**
+   * Write Client.
+   */
+  private transient HoodieFlinkWriteClient writeClient;
+
+  private Map<String, List<List<WriteStatus>>> bufferedWriteStatus = new HashMap<>();
+
+  private Integer upsertParallelSize = 0;
+
+  @Override
+  public void open(Configuration parameters) throws Exception {
+    super.open(parameters);
+    // get configs from runtimeContext
+    cfg = (HudiFlinkStreamer.Config) getRuntimeContext().getExecutionConfig().getGlobalJobParameters();
+    upsertParallelSize = getRuntimeContext().getExecutionConfig().getParallelism();
+
+    // HoodieWriteConfig
+    writeConfig = StreamerUtil.getHoodieClientConfig(cfg);
+
+    // writeClient
+    writeClient = new HoodieFlinkWriteClient<>(new HoodieFlinkEngineContext(new FlinkTaskContextSupplier(null)), writeConfig);
+  }
+
+  @Override
+  public void invoke(Tuple3<String, List<WriteStatus>, Integer> value, Context context) throws Exception {
+    LOG.info("Receive records, instantTime = [{}], subtaskId = [{}], records size = [{}]", value.f0, value.f2, value.f1.size());
+    try {
+      if (bufferedWriteStatus.containsKey(value.f0)) {
+        bufferedWriteStatus.get(value.f0).add(value.f1);
+      } else {
+        List<List<WriteStatus>> oneBatchData = new ArrayList<>(upsertParallelSize);
+        oneBatchData.add(value.f1);
+        bufferedWriteStatus.put(value.f0, oneBatchData);
+      }
+      // check and commit
+      checkAndCommit(value.f0);
+    } catch (Exception e) {
+      LOG.error("Invoke sink error: " + Thread.currentThread().getId() + ";" + this);
+      throw e;
+    }
+  }
+
+  /**
+   * Check and commit if all subtask completed.
+   *
+   * @throws Exception
+   */
+  private boolean checkAndCommit(String instantTime) throws Exception {

Review comment:
       You do not use the returned value, right?

##########
File path: hudi-flink-writer/src/main/java/org/apache/hudi/sink/CommitSink.java
##########
@@ -0,0 +1,150 @@
+/*
+ * 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.sink;
+
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
+import org.apache.hudi.HudiFlinkStreamer;
+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.util.Option;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.util.StreamerUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+public class CommitSink extends RichSinkFunction<Tuple3<String, List<WriteStatus>, Integer>> {
+  private static final Logger LOG = LoggerFactory.getLogger(CommitSink.class);
+  /**
+   * Job conf.
+   */
+  private HudiFlinkStreamer.Config cfg;
+
+  /**
+   * HoodieWriteConfig.
+   */
+  private HoodieWriteConfig writeConfig;
+
+  /**
+   * Write Client.
+   */
+  private transient HoodieFlinkWriteClient writeClient;
+
+  private Map<String, List<List<WriteStatus>>> bufferedWriteStatus = new HashMap<>();
+
+  private Integer upsertParallelSize = 0;
+
+  @Override
+  public void open(Configuration parameters) throws Exception {
+    super.open(parameters);
+    // get configs from runtimeContext
+    cfg = (HudiFlinkStreamer.Config) getRuntimeContext().getExecutionConfig().getGlobalJobParameters();
+    upsertParallelSize = getRuntimeContext().getExecutionConfig().getParallelism();
+
+    // HoodieWriteConfig
+    writeConfig = StreamerUtil.getHoodieClientConfig(cfg);
+
+    // writeClient
+    writeClient = new HoodieFlinkWriteClient<>(new HoodieFlinkEngineContext(new FlinkTaskContextSupplier(null)), writeConfig);
+  }
+
+  @Override
+  public void invoke(Tuple3<String, List<WriteStatus>, Integer> value, Context context) throws Exception {
+    LOG.info("Receive records, instantTime = [{}], subtaskId = [{}], records size = [{}]", value.f0, value.f2, value.f1.size());
+    try {
+      if (bufferedWriteStatus.containsKey(value.f0)) {
+        bufferedWriteStatus.get(value.f0).add(value.f1);
+      } else {
+        List<List<WriteStatus>> oneBatchData = new ArrayList<>(upsertParallelSize);
+        oneBatchData.add(value.f1);
+        bufferedWriteStatus.put(value.f0, oneBatchData);
+      }
+      // check and commit
+      checkAndCommit(value.f0);
+    } catch (Exception e) {
+      LOG.error("Invoke sink error: " + Thread.currentThread().getId() + ";" + this);
+      throw e;
+    }
+  }
+
+  /**
+   * Check and commit if all subtask completed.
+   *
+   * @throws Exception
+   */
+  private boolean checkAndCommit(String instantTime) throws Exception {
+    if (bufferedWriteStatus.get(instantTime).size() == upsertParallelSize) {
+      LOG.info("Instant [{}] process complete, start commit!", instantTime);
+      doCommit(instantTime);
+      bufferedWriteStatus.clear();
+      LOG.info("Instant [{}] commit completed!", instantTime);
+      return true;
+    } else {
+      LOG.info("Instant [{}], can not commit yet, subtask completed : [{}/{}]", instantTime, bufferedWriteStatus.get(instantTime).size(), upsertParallelSize);
+      return false;
+    }
+  }
+
+  public void doCommit(String instantTime) throws Exception {

Review comment:
       `public` -> `private`

##########
File path: hudi-flink-writer/src/main/java/org/apache/hudi/sink/CommitSink.java
##########
@@ -0,0 +1,150 @@
+/*
+ * 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.sink;
+
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
+import org.apache.hudi.HudiFlinkStreamer;
+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.util.Option;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.util.StreamerUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+public class CommitSink extends RichSinkFunction<Tuple3<String, List<WriteStatus>, Integer>> {
+  private static final Logger LOG = LoggerFactory.getLogger(CommitSink.class);
+  /**
+   * Job conf.
+   */
+  private HudiFlinkStreamer.Config cfg;
+
+  /**
+   * HoodieWriteConfig.
+   */
+  private HoodieWriteConfig writeConfig;
+
+  /**
+   * Write Client.
+   */
+  private transient HoodieFlinkWriteClient writeClient;
+
+  private Map<String, List<List<WriteStatus>>> bufferedWriteStatus = new HashMap<>();
+
+  private Integer upsertParallelSize = 0;
+
+  @Override
+  public void open(Configuration parameters) throws Exception {
+    super.open(parameters);
+    // get configs from runtimeContext
+    cfg = (HudiFlinkStreamer.Config) getRuntimeContext().getExecutionConfig().getGlobalJobParameters();
+    upsertParallelSize = getRuntimeContext().getExecutionConfig().getParallelism();
+
+    // HoodieWriteConfig
+    writeConfig = StreamerUtil.getHoodieClientConfig(cfg);
+
+    // writeClient
+    writeClient = new HoodieFlinkWriteClient<>(new HoodieFlinkEngineContext(new FlinkTaskContextSupplier(null)), writeConfig);
+  }
+
+  @Override
+  public void invoke(Tuple3<String, List<WriteStatus>, Integer> value, Context context) throws Exception {

Review comment:
       It would be better to make the `value` variable more reasonable.

##########
File path: hudi-flink-writer/src/main/java/org/apache/hudi/operator/WriteProcessOperator.java
##########
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.operator;
+
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.streaming.api.functions.KeyedProcessFunction;
+import org.apache.flink.streaming.api.operators.KeyedProcessOperator;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.List;
+
+public class WriteProcessOperator extends KeyedProcessOperator<String, HoodieRecord, Tuple3<String, List<WriteStatus>, Integer>> {

Review comment:
       WDYT about renaming to `KeyedWriteProcessOperator`? Since you use `KeyedWriteProcessFunction`.

##########
File path: hudi-flink-writer/src/main/java/org/apache/hudi/operator/KeyedWriteProcessFunction.java
##########
@@ -0,0 +1,164 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.operator;
+
+import org.apache.hudi.HudiFlinkStreamer;
+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.HoodieTableType;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.common.table.timeline.HoodieTimeline;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieFlinkStreamerException;
+import org.apache.hudi.util.StreamerUtil;
+
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.configuration.Configuration;
+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.util.Collector;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.LinkedList;
+import java.util.List;
+
+public class KeyedWriteProcessFunction extends KeyedProcessFunction<String, HoodieRecord, Tuple3<String, List<WriteStatus>, Integer>> implements CheckpointedFunction {
+
+  private static final Logger LOG = LoggerFactory.getLogger(KeyedWriteProcessFunction.class);
+  private List<HoodieRecord> records = new LinkedList<>();
+  private Collector<Tuple3<String, List<WriteStatus>, Integer>> output;
+  private int indexOfThisSubtask;
+  private String latestInstant;
+  private boolean hasRecordsIn;
+
+  /**
+   * Job conf.
+   */
+  private HudiFlinkStreamer.Config cfg;
+  /**
+   * Serializable hadoop conf.
+   */
+  private SerializableConfiguration serializableHadoopConf;
+  /**
+   * HoodieWriteConfig.
+   */
+  private HoodieWriteConfig writeConfig;
+
+  /**
+   * Write Client.
+   */
+  private transient HoodieFlinkWriteClient writeClient;
+
+  @Override
+  public void open(Configuration parameters) throws Exception {
+    super.open(parameters);
+
+    indexOfThisSubtask = getRuntimeContext().getIndexOfThisSubtask();
+
+    cfg = (HudiFlinkStreamer.Config) getRuntimeContext().getExecutionConfig().getGlobalJobParameters();
+
+    // hadoopConf
+    serializableHadoopConf = new SerializableConfiguration(new org.apache.hadoop.conf.Configuration());
+    // HoodieWriteConfig
+    writeConfig = StreamerUtil.getHoodieClientConfig(cfg);
+
+    HoodieFlinkEngineContext context = new HoodieFlinkEngineContext(serializableHadoopConf, new FlinkTaskContextSupplier(getRuntimeContext()));
+
+    // writeClient
+    writeClient = new HoodieFlinkWriteClient<>(context, writeConfig);
+  }
+
+  @Override
+  public void snapshotState(FunctionSnapshotContext context) throws Exception {

Review comment:
       Never thrown exception.




----------------------------------------------------------------
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 #2176: [HUDI-1327] Introduce base implementation of hudi-flink-client

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



##########
File path: hudi-flink-writer/src/main/java/org/apache/hudi/operator/WriteProcessOperator.java
##########
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.operator;
+
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.streaming.api.functions.KeyedProcessFunction;
+import org.apache.flink.streaming.api.operators.KeyedProcessOperator;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.List;
+
+public class WriteProcessOperator extends KeyedProcessOperator<String, HoodieRecord, Tuple3<String, List<WriteStatus>, Integer>> {

Review comment:
       > WDYT about renaming to `KeyedWriteProcessOperator`? Since you use `KeyedWriteProcessFunction`.
   
   yes, it's better




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] wangxianghu commented on a change in pull request #2176: [HUDI-1327] Introduce base implementation of hudi-flink-client

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



##########
File path: hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/upgrade/OneToZeroDowngradeHandler.java
##########
@@ -0,0 +1,49 @@
+/*
+ * 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.table.upgrade;
+
+import org.apache.hudi.client.common.HoodieEngineContext;
+import org.apache.hudi.client.common.HoodieFlinkEngineContext;
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.common.table.timeline.HoodieTimeline;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.table.HoodieFlinkTable;
+import org.apache.hudi.table.MarkerFiles;
+
+import java.util.List;
+import java.util.stream.Collectors;
+
+/**
+ * Downgrade handle to assist in downgrading hoodie table from version 1 to 0.

Review comment:
       > what is the diff between version 1 and 0?
   
   0  - hudi version < 0.6.0
   1 - hudi version >= 0.6.0




----------------------------------------------------------------
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 #2176: [HUDI-1327] Introduce base implementation of hudi-flink-client

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



##########
File path: hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/HoodieFlinkWriteClient.java
##########
@@ -0,0 +1,219 @@
+/*
+ * 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.client;
+
+import org.apache.hudi.client.common.HoodieEngineContext;
+import org.apache.hudi.client.common.HoodieFlinkEngineContext;
+import org.apache.hudi.client.embedded.EmbeddedTimelineService;
+import org.apache.hudi.common.model.HoodieCommitMetadata;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.common.model.HoodieWriteStat;
+import org.apache.hudi.common.model.WriteOperationType;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.HoodieTableVersion;
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.common.table.timeline.HoodieTimeline;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.index.FlinkHoodieIndex;
+import org.apache.hudi.index.HoodieIndex;
+import org.apache.hudi.table.BulkInsertPartitioner;
+import org.apache.hudi.table.HoodieFlinkTable;
+import org.apache.hudi.table.HoodieTable;
+import org.apache.hudi.table.action.HoodieWriteMetadata;
+import org.apache.hudi.table.upgrade.FlinkUpgradeDowngrade;
+
+import com.codahale.metrics.Timer;
+import org.apache.hadoop.conf.Configuration;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+@SuppressWarnings("checkstyle:LineLength")
+public class HoodieFlinkWriteClient<T extends HoodieRecordPayload> extends
+    AbstractHoodieWriteClient<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> {
+
+  public HoodieFlinkWriteClient(HoodieEngineContext context, HoodieWriteConfig clientConfig) {
+    super(context, clientConfig);
+  }
+
+  public HoodieFlinkWriteClient(HoodieEngineContext context, HoodieWriteConfig writeConfig, boolean rollbackPending) {

Review comment:
       used 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] codecov-io edited a comment on pull request #2176: [HUDI-1327] Introduce base implementation of hudi-flink-client

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


   # [Codecov](https://codecov.io/gh/apache/hudi/pull/2176?src=pr&el=h1) Report
   > Merging [#2176](https://codecov.io/gh/apache/hudi/pull/2176?src=pr&el=desc) (83d7a0d) into [master](https://codecov.io/gh/apache/hudi/commit/0364498ae3be553f7c9fab78469283a6958da2ff?el=desc) (0364498) will **decrease** coverage by `43.13%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/hudi/pull/2176/graphs/tree.svg?width=650&height=150&src=pr&token=VTTXabwbs2)](https://codecov.io/gh/apache/hudi/pull/2176?src=pr&el=tree)
   
   ```diff
   @@              Coverage Diff              @@
   ##             master    #2176       +/-   ##
   =============================================
   - Coverage     53.53%   10.39%   -43.14%     
   + Complexity     2770       48     -2722     
   =============================================
     Files           348       50      -298     
     Lines         16099     1779    -14320     
     Branches       1642      211     -1431     
   =============================================
   - Hits           8619      185     -8434     
   + Misses         6785     1581     -5204     
   + Partials        695       13      -682     
   ```
   
   | Flag | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | hudicli | `?` | `?` | |
   | hudiclient | `?` | `?` | |
   | hudicommon | `?` | `?` | |
   | hudihadoopmr | `?` | `?` | |
   | hudispark | `?` | `?` | |
   | huditimelineservice | `?` | `?` | |
   | hudiutilities | `10.39% <ø> (-59.70%)` | `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/2176?src=pr&el=tree) | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | [...va/org/apache/hudi/utilities/IdentitySplitter.java](https://codecov.io/gh/apache/hudi/pull/2176/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/2176/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/2176/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/2176/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/2176/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/2176/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/2176/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/2176/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/2176/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/2176/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NjaGVtYS9TY2hlbWFQcm92aWRlcldpdGhQb3N0UHJvY2Vzc29yLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-3.00%)` | |
   | ... and [325 more](https://codecov.io/gh/apache/hudi/pull/2176/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] wangxianghu commented on a change in pull request #2176: [HUDI-1327] Introduce base implementation of hudi-flink-client

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



##########
File path: hudi-flink-writer/src/main/java/org/apache/hudi/operator/InstantGenerateOperator.java
##########
@@ -0,0 +1,219 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.operator;
+
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hudi.HudiFlinkStreamer;
+import org.apache.hudi.client.FlinkTaskContextSupplier;
+import org.apache.hudi.client.HoodieFlinkWriteClient;
+import org.apache.hudi.client.common.HoodieFlinkEngineContext;
+import org.apache.hudi.client.common.TaskContextSupplier;
+import org.apache.hudi.common.config.SerializableConfiguration;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieTableType;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.timeline.HoodieTimeline;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.util.StreamerUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+
+/**
+ * Operator helps to generate globally unique instant, it must be executed in one parallelism. Before generate a new
+ * instant , {@link InstantGenerateOperator} will always check whether the last instant has completed. if it is
+ * completed, a new instant will be generated immediately, otherwise, wait and check the state of last instant until
+ * time out and throw an exception.
+ */
+public class InstantGenerateOperator extends AbstractStreamOperator<HoodieRecord> implements OneInputStreamOperator<HoodieRecord, HoodieRecord> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(InstantGenerateOperator.class);
+  public static final String NAME = "InstantGenerateOperator";
+
+  private HudiFlinkStreamer.Config cfg;
+  private HoodieFlinkWriteClient writeClient;
+  private SerializableConfiguration serializableHadoopConf;
+  private transient FileSystem fs;
+  private String latestInstant = "";
+  private List<String> latestInstantList = new ArrayList<>(1);
+  private transient ListState<String> latestInstantState;
+  private List<StreamRecord> bufferedRecords = new LinkedList();
+  private transient ListState<StreamRecord> recordsState;
+  private Integer commitTimeout;
+
+  @Override
+  public void processElement(StreamRecord<HoodieRecord> streamRecord) throws Exception {
+    if (streamRecord.getValue() != null) {
+      bufferedRecords.add(streamRecord);
+      output.collect(streamRecord);
+    }
+  }
+
+  @Override
+  public void open() throws Exception {
+    super.open();
+    // get configs from runtimeContext
+    cfg = (HudiFlinkStreamer.Config) getRuntimeContext().getExecutionConfig().getGlobalJobParameters();
+
+    // timeout
+    commitTimeout = Integer.valueOf(cfg.flinkCommitTimeout);
+
+    // hadoopConf
+    serializableHadoopConf = new SerializableConfiguration(StreamerUtil.getHadoopConf());
+
+    // Hadoop FileSystem
+    fs = FSUtils.getFs(cfg.targetBasePath, serializableHadoopConf.get());
+
+    TaskContextSupplier taskContextSupplier = new FlinkTaskContextSupplier(null);
+
+    // writeClient
+    writeClient = new HoodieFlinkWriteClient(new HoodieFlinkEngineContext(taskContextSupplier), StreamerUtil.getHoodieClientConfig(cfg));
+
+    // init table, create it if not exists.
+    initTable();
+  }
+
+  @Override
+  public void prepareSnapshotPreBarrier(long checkpointId) throws Exception {
+    super.prepareSnapshotPreBarrier(checkpointId);
+    // check whether the last instant is completed, if not, wait 10s and then throws an exception
+    if (!StringUtils.isNullOrEmpty(latestInstant)) {
+      doChecker();
+      // last instant completed, set it empty
+      latestInstant = "";
+    }
+
+    // no data no new instant
+    if (!bufferedRecords.isEmpty()) {
+      latestInstant = startNewInstant(checkpointId);
+    }
+  }
+
+  @Override
+  public void initializeState(StateInitializationContext context) throws Exception {
+    // instantState
+    ListStateDescriptor<String> latestInstantStateDescriptor = new ListStateDescriptor<String>("latestInstant", String.class);
+    latestInstantState = context.getOperatorStateStore().getListState(latestInstantStateDescriptor);
+
+    // recordState
+    ListStateDescriptor<StreamRecord> recordsStateDescriptor = new ListStateDescriptor<StreamRecord>("recordsState", StreamRecord.class);
+    recordsState = context.getOperatorStateStore().getListState(recordsStateDescriptor);
+
+    if (context.isRestored()) {
+      Iterator<String> latestInstantIterator = latestInstantState.get().iterator();
+      latestInstantIterator.forEachRemaining(x -> latestInstant = x);
+      LOG.info("InstantGenerateOperator initializeState get latestInstant [{}]", latestInstant);
+
+      Iterator<StreamRecord> recordIterator = recordsState.get().iterator();
+      bufferedRecords.clear();
+      recordIterator.forEachRemaining(x -> bufferedRecords.add(x));
+    }
+  }
+
+  @Override
+  public void snapshotState(StateSnapshotContext functionSnapshotContext) throws Exception {
+    if (latestInstantList.isEmpty()) {
+      latestInstantList.add(latestInstant);
+    } else {
+      latestInstantList.set(0, latestInstant);
+    }
+    latestInstantState.update(latestInstantList);
+    LOG.info("Update latest instant [{}]", latestInstant);
+
+    recordsState.update(bufferedRecords);
+    LOG.info("Update records state size = [{}]", bufferedRecords.size());
+    bufferedRecords.clear();
+  }
+
+  /**
+   * Create a new instant.
+   *
+   * @param checkpointId
+   */
+  private String startNewInstant(long checkpointId) {
+    String newTime = writeClient.startCommit();
+    LOG.info("create instant [{}], at checkpoint [{}]", newTime, checkpointId);
+    return newTime;
+  }
+
+  /**
+   * Check the status of last instant.
+   */
+  private void doChecker() throws InterruptedException {

Review comment:
       done




----------------------------------------------------------------
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 #2176: [HUDI-1327] Introduce base implementation of hudi-flink-client

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


   @wangxianghu Please see my apporve information.


----------------------------------------------------------------
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 #2176: [HUDI-1327] Introduce base implementation of hudi-flink-client

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



##########
File path: hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/index/state/FlinkInMemoryStateIndex.java
##########
@@ -0,0 +1,135 @@
+/*
+ * 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.index.state;
+
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.client.common.HoodieEngineContext;
+import org.apache.hudi.client.common.HoodieFlinkEngineContext;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordLocation;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieIndexException;
+import org.apache.hudi.index.FlinkHoodieIndex;
+import org.apache.hudi.table.HoodieTable;
+
+import org.apache.flink.api.common.state.MapState;
+import org.apache.flink.api.common.state.MapStateDescriptor;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.util.List;
+
+/**
+ * Hoodie index implementation backed by flink state.
+ *
+ * @param <T> type of payload
+ */
+public class FlinkInMemoryStateIndex<T extends HoodieRecordPayload> extends FlinkHoodieIndex<T> {

Review comment:
       > @wangxianghu sounds good. please file a JIRA if there is not one already.
   
   filed here : https://issues.apache.org/jira/browse/HUDI-981




----------------------------------------------------------------
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 #2176: [HUDI-1327] Introduce base implementation of hudi-flink-client

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



##########
File path: hudi-flink-writer/src/main/java/org/apache/hudi/schema/FilebasedSchemaProvider.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.schema;
+
+import org.apache.avro.Schema;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.exception.HoodieIOException;
+import org.apache.hudi.util.StreamerUtil;
+
+import java.io.IOException;
+import java.util.Collections;
+
+/**
+ * A simple schema provider, that reads off files on DFS.
+ */
+public class FilebasedSchemaProvider extends SchemaProvider {

Review comment:
       > we have `TableSchemaResolver` to handle the schema
   
   `TableSchemaResolver`  helps to read schema from data files and log files, here we need a Util to read schema from job config




----------------------------------------------------------------
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 #2176: [HUDI-1327] Introduce base implementation of hudi-flink-client

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



##########
File path: hudi-flink-writer/src/main/java/org/apache/hudi/operator/InstantGenerateOperator.java
##########
@@ -0,0 +1,219 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.operator;
+
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hudi.HudiFlinkStreamer;
+import org.apache.hudi.client.FlinkTaskContextSupplier;
+import org.apache.hudi.client.HoodieFlinkWriteClient;
+import org.apache.hudi.client.common.HoodieFlinkEngineContext;
+import org.apache.hudi.client.common.TaskContextSupplier;
+import org.apache.hudi.common.config.SerializableConfiguration;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieTableType;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.timeline.HoodieTimeline;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.util.StreamerUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+
+/**
+ * Operator helps to generate globally unique instant, it must be executed in one parallelism. Before generate a new
+ * instant , {@link InstantGenerateOperator} will always check whether the last instant has completed. if it is
+ * completed, a new instant will be generated immediately, otherwise, wait and check the state of last instant until
+ * time out and throw an exception.
+ */
+public class InstantGenerateOperator extends AbstractStreamOperator<HoodieRecord> implements OneInputStreamOperator<HoodieRecord, HoodieRecord> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(InstantGenerateOperator.class);
+  public static final String NAME = "InstantGenerateOperator";
+
+  private HudiFlinkStreamer.Config cfg;
+  private HoodieFlinkWriteClient writeClient;
+  private SerializableConfiguration serializableHadoopConf;
+  private transient FileSystem fs;
+  private String latestInstant = "";
+  private List<String> latestInstantList = new ArrayList<>(1);
+  private transient ListState<String> latestInstantState;
+  private List<StreamRecord> bufferedRecords = new LinkedList();
+  private transient ListState<StreamRecord> recordsState;
+  private Integer commitTimeout;
+
+  @Override
+  public void processElement(StreamRecord<HoodieRecord> streamRecord) throws Exception {
+    if (streamRecord.getValue() != null) {
+      bufferedRecords.add(streamRecord);
+      output.collect(streamRecord);
+    }
+  }
+
+  @Override
+  public void open() throws Exception {
+    super.open();
+    // get configs from runtimeContext
+    cfg = (HudiFlinkStreamer.Config) getRuntimeContext().getExecutionConfig().getGlobalJobParameters();
+
+    // timeout
+    commitTimeout = Integer.valueOf(cfg.flinkCommitTimeout);
+
+    // hadoopConf
+    serializableHadoopConf = new SerializableConfiguration(StreamerUtil.getHadoopConf());
+
+    // Hadoop FileSystem
+    fs = FSUtils.getFs(cfg.targetBasePath, serializableHadoopConf.get());
+
+    TaskContextSupplier taskContextSupplier = new FlinkTaskContextSupplier(null);
+
+    // writeClient
+    writeClient = new HoodieFlinkWriteClient(new HoodieFlinkEngineContext(taskContextSupplier), StreamerUtil.getHoodieClientConfig(cfg));
+
+    // init table, create it if not exists.
+    initTable();
+  }
+
+  @Override
+  public void prepareSnapshotPreBarrier(long checkpointId) throws Exception {
+    super.prepareSnapshotPreBarrier(checkpointId);
+    // check whether the last instant is completed, if not, wait 10s and then throws an exception
+    if (!StringUtils.isNullOrEmpty(latestInstant)) {
+      doChecker();
+      // last instant completed, set it empty
+      latestInstant = "";
+    }
+
+    // no data no new instant
+    if (!bufferedRecords.isEmpty()) {
+      latestInstant = startNewInstant(checkpointId);
+    }
+  }
+
+  @Override
+  public void initializeState(StateInitializationContext context) throws Exception {
+    // instantState
+    ListStateDescriptor<String> latestInstantStateDescriptor = new ListStateDescriptor<String>("latestInstant", String.class);
+    latestInstantState = context.getOperatorStateStore().getListState(latestInstantStateDescriptor);
+
+    // recordState
+    ListStateDescriptor<StreamRecord> recordsStateDescriptor = new ListStateDescriptor<StreamRecord>("recordsState", StreamRecord.class);
+    recordsState = context.getOperatorStateStore().getListState(recordsStateDescriptor);
+
+    if (context.isRestored()) {
+      Iterator<String> latestInstantIterator = latestInstantState.get().iterator();
+      latestInstantIterator.forEachRemaining(x -> latestInstant = x);
+      LOG.info("InstantGenerateOperator initializeState get latestInstant [{}]", latestInstant);
+
+      Iterator<StreamRecord> recordIterator = recordsState.get().iterator();
+      bufferedRecords.clear();
+      recordIterator.forEachRemaining(x -> bufferedRecords.add(x));
+    }
+  }
+
+  @Override
+  public void snapshotState(StateSnapshotContext functionSnapshotContext) throws Exception {
+    if (latestInstantList.isEmpty()) {
+      latestInstantList.add(latestInstant);
+    } else {
+      latestInstantList.set(0, latestInstant);
+    }
+    latestInstantState.update(latestInstantList);
+    LOG.info("Update latest instant [{}]", latestInstant);
+
+    recordsState.update(bufferedRecords);
+    LOG.info("Update records state size = [{}]", bufferedRecords.size());
+    bufferedRecords.clear();
+  }
+
+  /**
+   * Create a new instant.
+   *
+   * @param checkpointId
+   */
+  private String startNewInstant(long checkpointId) {
+    String newTime = writeClient.startCommit();
+    LOG.info("create instant [{}], at checkpoint [{}]", newTime, checkpointId);
+    return newTime;
+  }
+
+  /**
+   * Check the status of last instant.
+   */
+  private void doChecker() throws InterruptedException {
+    // query the requested and inflight commit/deltacommit instants
+    String commitType = cfg.tableType.equals(HoodieTableType.COPY_ON_WRITE.name()) ? HoodieTimeline.COMMIT_ACTION : HoodieTimeline.DELTA_COMMIT_ACTION;
+    LOG.info("Query latest instant [{}]", latestInstant);
+    List<String> rollbackPendingCommits = writeClient.getInflightsAndRequestedInstants(commitType);
+    int tryTimes = 0;
+    while (tryTimes < commitTimeout) {

Review comment:
       > IMO, we can introduce both `retry number` and `timeout` conditions. WDYT?
   
   good idea




----------------------------------------------------------------
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] leesf commented on a change in pull request #2176: [HUDI-1327] Introduce base implementation of hudi-flink-client

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



##########
File path: hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/FlinkTaskContextSupplier.java
##########
@@ -0,0 +1,65 @@
+/*
+ * 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.client;
+
+import org.apache.hudi.client.common.EngineProperty;
+import org.apache.hudi.client.common.TaskContextSupplier;
+import org.apache.hudi.common.util.Option;
+
+import org.apache.flink.api.common.functions.RuntimeContext;
+
+import java.util.function.Supplier;
+
+/**
+ * Flink task context supplier.
+ */
+public class FlinkTaskContextSupplier extends TaskContextSupplier {
+  private org.apache.flink.api.common.functions.RuntimeContext flinkRuntimeContext;
+
+  public FlinkTaskContextSupplier(RuntimeContext flinkRuntimeContext) {
+    this.flinkRuntimeContext = flinkRuntimeContext;
+  }
+
+  public RuntimeContext getFlinkRuntimeContext() {
+    return flinkRuntimeContext;
+  }
+
+  @Override
+  public Supplier<Integer> getPartitionIdSupplier() {
+    return () -> this.flinkRuntimeContext.getIndexOfThisSubtask();
+  }
+
+  @Override
+  public Supplier<Integer> getStageIdSupplier() {
+    // need to check again

Review comment:
       // TODO need to check again?




----------------------------------------------------------------
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 #2176: [HUDI-1327] Introduce base implementation of hudi-flink-client

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



##########
File path: hudi-flink-writer/src/main/java/org/apache/hudi/schema/FilebasedSchemaProvider.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.schema;
+
+import org.apache.avro.Schema;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.exception.HoodieIOException;
+import org.apache.hudi.util.StreamerUtil;
+
+import java.io.IOException;
+import java.util.Collections;
+
+/**
+ * A simple schema provider, that reads off files on DFS.
+ */
+public class FilebasedSchemaProvider extends SchemaProvider {

Review comment:
       > ah ok, I got confused by the class name. Should we move this to `TableSchemaResolver` then, so we have schema related stuff in one place. WDYT?
   
   Yes, keep schema related stuff together makes sense. 
   Maybe we can leave `TableSchemaResolver` where it is and move `SchemaProvider` stuff to `hudi-client-common` , because `TableSchemaResolver`  basically will not be modified in the future, while `SchemaProvider`  is expansion-oriented, more likely to add new child implementation, and more like client stuff.
   
   besides, this `SchemaProvider` comes from `hudi-utilities` module, and there are still lots of utils can be reused there, we can file a new ticket to do things like this together.
   WDYT?




----------------------------------------------------------------
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 #2176: [HUDI-1327] Introduce base implementation of hudi-flink-client

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



##########
File path: hudi-flink-writer/src/main/java/org/apache/hudi/operator/InstantGenerateOperator.java
##########
@@ -0,0 +1,208 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.operator;
+
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hudi.HudiFlinkStreamer;
+import org.apache.hudi.client.FlinkTaskContextSupplier;
+import org.apache.hudi.client.HoodieFlinkWriteClient;
+import org.apache.hudi.client.common.HoodieFlinkEngineContext;
+import org.apache.hudi.client.common.TaskContextSupplier;
+import org.apache.hudi.common.config.SerializableConfiguration;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieTableType;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.timeline.HoodieTimeline;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.util.StreamerUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+
+public class InstantGenerateOperator extends AbstractStreamOperator<HoodieRecord> implements OneInputStreamOperator<HoodieRecord, HoodieRecord> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(InstantGenerateOperator.class);
+  public static final String NAME = "InstantGenerateOperator";

Review comment:
       done




----------------------------------------------------------------
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 #2176: [HUDI-1327] Introduce base implementation of hudi-flink-client

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



##########
File path: hudi-flink-writer/src/main/java/org/apache/hudi/source/KafkaJson2HoodieRecord.java
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.source;
+
+import org.apache.avro.generic.GenericRecord;
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.hudi.HudiFlinkStreamer;
+import org.apache.hudi.avro.HoodieAvroUtils;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.keygen.KeyGenerator;
+import org.apache.hudi.schema.FilebasedSchemaProvider;
+import org.apache.hudi.util.AvroConvertor;
+import org.apache.hudi.util.StreamerUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+public class KafkaJson2HoodieRecord implements MapFunction<String, HoodieRecord> {

Review comment:
       > It would be better to add `xxxFunction` as the suffix. wdyt?
   yes , it is better




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] garyli1019 commented on a change in pull request #2176: [HUDI-1327] Introduce base implementation of hudi-flink-client

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



##########
File path: hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/upgrade/OneToZeroDowngradeHandler.java
##########
@@ -0,0 +1,49 @@
+/*
+ * 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.table.upgrade;
+
+import org.apache.hudi.client.common.HoodieEngineContext;
+import org.apache.hudi.client.common.HoodieFlinkEngineContext;
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.common.table.timeline.HoodieTimeline;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.table.HoodieFlinkTable;
+import org.apache.hudi.table.MarkerFiles;
+
+import java.util.List;
+import java.util.stream.Collectors;
+
+/**
+ * Downgrade handle to assist in downgrading hoodie table from version 1 to 0.

Review comment:
       what is the diff between version 1 and 0?

##########
File path: hudi-flink-writer/src/main/java/org/apache/hudi/HudiFlinkStreamer.java
##########
@@ -0,0 +1,186 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi;
+
+import com.beust.jcommander.IStringConverter;
+import com.beust.jcommander.JCommander;
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.ParameterException;
+import org.apache.flink.api.common.serialization.SimpleStringSchema;
+import org.apache.flink.api.common.typeinfo.TypeHint;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.state.filesystem.FsStateBackend;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload;
+import org.apache.hudi.constant.Operation;
+import org.apache.hudi.operator.InstantGenerateOperator;
+import org.apache.hudi.operator.KeyedWriteProcessFunction;
+import org.apache.hudi.operator.KeyedWriteProcessOperator;
+import org.apache.hudi.sink.CommitSink;
+import org.apache.hudi.source.JsonStringToHoodieRecordMapFunction;
+import org.apache.hudi.util.StreamerUtil;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Properties;
+
+/**
+ * An Utility which can incrementally consume data from Kafka and apply it to the target table.
+ * currently, it only support MOR table and insert, upsert operation.
+ */
+public class HudiFlinkStreamer {
+  public static void main(String[] args) throws Exception {
+    StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+
+    final Config cfg = new Config();
+    JCommander cmd = new JCommander(cfg, null, args);
+    if (cfg.help || args.length == 0) {
+      cmd.usage();
+      System.exit(1);
+    }
+    env.enableCheckpointing(cfg.checkpointInterval);
+    env.getConfig().setGlobalJobParameters(cfg);
+    // 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);
+
+    // Read from kafka source
+    DataStream<HoodieRecord> inputRecords =
+        env.addSource(new FlinkKafkaConsumer<>(cfg.kafkaTopic, new SimpleStringSchema(), kafkaProps))
+            .map(new JsonStringToHoodieRecordMapFunction(cfg))
+            .name("kafka_to_hudi_record")
+            .uid("kafka_to_hudi_record_uid");
+
+    // InstantGenerateOperator helps to emit globally unique instantTime, it must be executed in one parallelism
+    inputRecords.transform(InstantGenerateOperator.NAME, TypeInformation.of(HoodieRecord.class), new InstantGenerateOperator())
+        .name("instant_generator")
+        .uid("instant_generator_id")
+        .setParallelism(1)

Review comment:
       would you add a bit more comments here? not quite familiar with Flink but a bit worried about if this will impact the performance

##########
File path: hudi-flink-writer/src/main/java/org/apache/hudi/schema/FilebasedSchemaProvider.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.schema;
+
+import org.apache.avro.Schema;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.exception.HoodieIOException;
+import org.apache.hudi.util.StreamerUtil;
+
+import java.io.IOException;
+import java.util.Collections;
+
+/**
+ * A simple schema provider, that reads off files on DFS.
+ */
+public class FilebasedSchemaProvider extends SchemaProvider {

Review comment:
       we have `TableSchemaResolver` to handle the schema

##########
File path: hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/index/state/FlinkInMemoryStateIndex.java
##########
@@ -0,0 +1,135 @@
+/*
+ * 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.index.state;
+
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.client.common.HoodieEngineContext;
+import org.apache.hudi.client.common.HoodieFlinkEngineContext;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordLocation;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieIndexException;
+import org.apache.hudi.index.FlinkHoodieIndex;
+import org.apache.hudi.table.HoodieTable;
+
+import org.apache.flink.api.common.state.MapState;
+import org.apache.flink.api.common.state.MapStateDescriptor;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.util.List;
+
+/**
+ * Hoodie index implementation backed by flink state.
+ *
+ * @param <T> type of payload
+ */
+public class FlinkInMemoryStateIndex<T extends HoodieRecordPayload> extends FlinkHoodieIndex<T> {

Review comment:
       Is this only dedup data in one batch but not upserting into the historical hudi table? 
   




----------------------------------------------------------------
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 #2176: [HUDI-1327] Introduce base implementation of hudi-flink-client

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



##########
File path: hudi-flink-writer/src/main/java/org/apache/hudi/constant/Operation.java
##########
@@ -0,0 +1,23 @@
+/*
+ * 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.constant;
+
+public enum Operation {
+  UPSERT, INSERT

Review comment:
       > can we reuse WriteOperationType, that already exists in hudi-common?
   
   sure, will do tonight




----------------------------------------------------------------
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 pull request #2176: [HUDI-1327] Introduce base implementation of hudi-flink-client

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


   > @wangxianghu big congrats! looking forward to the upcoming flink integration works
   
   
   > LGTM. It's a big and continuous work. Merge it now, hope you would continuously improve and enhance the flink-write-client.
   
   Sure, I will enhance it step by step
   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] leesf commented on a change in pull request #2176: [HUDI-1327] Introduce base implementation of hudi-flink-client

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



##########
File path: hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/index/FlinkHoodieIndex.java
##########
@@ -0,0 +1,90 @@
+/*
+ * 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.index;
+
+import org.apache.hudi.ApiMaturityLevel;
+import org.apache.hudi.PublicAPIMethod;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.client.common.HoodieEngineContext;
+import org.apache.hudi.client.common.HoodieFlinkEngineContext;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.common.util.ReflectionUtils;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieIndexException;
+import org.apache.hudi.index.state.FlinkInMemoryStateIndex;
+import org.apache.hudi.table.HoodieTable;
+
+import java.util.List;
+
+/**
+ * Base flink implementation of {@link HoodieIndex}.
+ * @param <T> payload type
+ */
+public abstract class FlinkHoodieIndex<T extends HoodieRecordPayload> extends HoodieIndex<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> {
+  protected FlinkHoodieIndex(HoodieWriteConfig config) {
+    super(config);
+  }
+
+  public static FlinkHoodieIndex createIndex(HoodieFlinkEngineContext context, HoodieWriteConfig config) {
+    // first use index class config to create index.
+    if (!StringUtils.isNullOrEmpty(config.getIndexClass())) {
+      Object instance = ReflectionUtils.loadClass(config.getIndexClass(), config);
+      if (!(instance instanceof HoodieIndex)) {
+        throw new HoodieIndexException(config.getIndexClass() + " is not a subclass of HoodieIndex");
+      }
+      return (FlinkHoodieIndex) instance;
+    }
+    switch (config.getIndexType()) {
+      case HBASE:
+        // TODO
+        return null;
+      case INMEMORY:
+        return new FlinkInMemoryStateIndex<>(context, config);
+      case BLOOM:
+        // TODO
+        return null;
+      case GLOBAL_BLOOM:
+        // TODO
+        return null;
+      case SIMPLE:
+        // TODO
+        return null;
+      case GLOBAL_SIMPLE:
+        // TODO
+        return null;

Review comment:
       should we implement some index 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] garyli1019 commented on pull request #2176: [HUDI-1327] Introduce base implementation of hudi-flink-client

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


   @wangxianghu big congrats! looking forward to the upcoming flink integration works


----------------------------------------------------------------
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 #2176: [HUDI-1327] Introduce base implementation of hudi-flink-client

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



##########
File path: hudi-flink-writer/src/main/java/org/apache/hudi/operator/InstantGenerateOperator.java
##########
@@ -0,0 +1,219 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.operator;
+
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hudi.HudiFlinkStreamer;
+import org.apache.hudi.client.FlinkTaskContextSupplier;
+import org.apache.hudi.client.HoodieFlinkWriteClient;
+import org.apache.hudi.client.common.HoodieFlinkEngineContext;
+import org.apache.hudi.client.common.TaskContextSupplier;
+import org.apache.hudi.common.config.SerializableConfiguration;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieTableType;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.timeline.HoodieTimeline;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.util.StreamerUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+
+/**
+ * Operator helps to generate globally unique instant, it must be executed in one parallelism. Before generate a new
+ * instant , {@link InstantGenerateOperator} will always check whether the last instant has completed. if it is
+ * completed, a new instant will be generated immediately, otherwise, wait and check the state of last instant until
+ * time out and throw an exception.
+ */
+public class InstantGenerateOperator extends AbstractStreamOperator<HoodieRecord> implements OneInputStreamOperator<HoodieRecord, HoodieRecord> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(InstantGenerateOperator.class);
+  public static final String NAME = "InstantGenerateOperator";
+
+  private HudiFlinkStreamer.Config cfg;
+  private HoodieFlinkWriteClient writeClient;
+  private SerializableConfiguration serializableHadoopConf;
+  private transient FileSystem fs;
+  private String latestInstant = "";
+  private List<String> latestInstantList = new ArrayList<>(1);
+  private transient ListState<String> latestInstantState;
+  private List<StreamRecord> bufferedRecords = new LinkedList();
+  private transient ListState<StreamRecord> recordsState;
+  private Integer commitTimeout;
+
+  @Override
+  public void processElement(StreamRecord<HoodieRecord> streamRecord) throws Exception {
+    if (streamRecord.getValue() != null) {
+      bufferedRecords.add(streamRecord);
+      output.collect(streamRecord);
+    }
+  }
+
+  @Override
+  public void open() throws Exception {
+    super.open();
+    // get configs from runtimeContext
+    cfg = (HudiFlinkStreamer.Config) getRuntimeContext().getExecutionConfig().getGlobalJobParameters();
+
+    // timeout
+    commitTimeout = Integer.valueOf(cfg.flinkCommitTimeout);
+
+    // hadoopConf
+    serializableHadoopConf = new SerializableConfiguration(StreamerUtil.getHadoopConf());
+
+    // Hadoop FileSystem
+    fs = FSUtils.getFs(cfg.targetBasePath, serializableHadoopConf.get());
+
+    TaskContextSupplier taskContextSupplier = new FlinkTaskContextSupplier(null);
+
+    // writeClient
+    writeClient = new HoodieFlinkWriteClient(new HoodieFlinkEngineContext(taskContextSupplier), StreamerUtil.getHoodieClientConfig(cfg));
+
+    // init table, create it if not exists.
+    initTable();
+  }
+
+  @Override
+  public void prepareSnapshotPreBarrier(long checkpointId) throws Exception {
+    super.prepareSnapshotPreBarrier(checkpointId);
+    // check whether the last instant is completed, if not, wait 10s and then throws an exception
+    if (!StringUtils.isNullOrEmpty(latestInstant)) {
+      doChecker();
+      // last instant completed, set it empty
+      latestInstant = "";
+    }
+
+    // no data no new instant
+    if (!bufferedRecords.isEmpty()) {
+      latestInstant = startNewInstant(checkpointId);
+    }
+  }
+
+  @Override
+  public void initializeState(StateInitializationContext context) throws Exception {
+    // instantState
+    ListStateDescriptor<String> latestInstantStateDescriptor = new ListStateDescriptor<String>("latestInstant", String.class);
+    latestInstantState = context.getOperatorStateStore().getListState(latestInstantStateDescriptor);
+
+    // recordState
+    ListStateDescriptor<StreamRecord> recordsStateDescriptor = new ListStateDescriptor<StreamRecord>("recordsState", StreamRecord.class);
+    recordsState = context.getOperatorStateStore().getListState(recordsStateDescriptor);
+
+    if (context.isRestored()) {
+      Iterator<String> latestInstantIterator = latestInstantState.get().iterator();
+      latestInstantIterator.forEachRemaining(x -> latestInstant = x);
+      LOG.info("InstantGenerateOperator initializeState get latestInstant [{}]", latestInstant);
+
+      Iterator<StreamRecord> recordIterator = recordsState.get().iterator();
+      bufferedRecords.clear();
+      recordIterator.forEachRemaining(x -> bufferedRecords.add(x));
+    }
+  }
+
+  @Override
+  public void snapshotState(StateSnapshotContext functionSnapshotContext) throws Exception {
+    if (latestInstantList.isEmpty()) {
+      latestInstantList.add(latestInstant);
+    } else {
+      latestInstantList.set(0, latestInstant);
+    }
+    latestInstantState.update(latestInstantList);
+    LOG.info("Update latest instant [{}]", latestInstant);
+
+    recordsState.update(bufferedRecords);
+    LOG.info("Update records state size = [{}]", bufferedRecords.size());
+    bufferedRecords.clear();
+  }
+
+  /**
+   * Create a new instant.
+   *
+   * @param checkpointId
+   */
+  private String startNewInstant(long checkpointId) {
+    String newTime = writeClient.startCommit();
+    LOG.info("create instant [{}], at checkpoint [{}]", newTime, checkpointId);
+    return newTime;
+  }
+
+  /**
+   * Check the status of last instant.
+   */
+  private void doChecker() throws InterruptedException {
+    // query the requested and inflight commit/deltacommit instants
+    String commitType = cfg.tableType.equals(HoodieTableType.COPY_ON_WRITE.name()) ? HoodieTimeline.COMMIT_ACTION : HoodieTimeline.DELTA_COMMIT_ACTION;
+    LOG.info("Query latest instant [{}]", latestInstant);
+    List<String> rollbackPendingCommits = writeClient.getInflightsAndRequestedInstants(commitType);
+    int tryTimes = 0;
+    while (tryTimes < commitTimeout) {
+      tryTimes++;
+      StringBuffer sb = new StringBuffer();
+      if (rollbackPendingCommits.contains(latestInstant)) {
+        //清空 sb

Review comment:
       > Chinese? Why this is a clear action to string buffer?
   
   my bad




----------------------------------------------------------------
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 pull request #2176: [HUDI-1327] Introduce base implementation of hudi-flink-client

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


   Hi @vinothchandar @yanghua @leesf @garyli1019 The ci is green now, please take a look when free


----------------------------------------------------------------
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 #2176: [HUDI-1327] Introduce base implementation of hudi-flink-client

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



##########
File path: hudi-flink-writer/src/main/java/org/apache/hudi/operator/InstantGenerateOperator.java
##########
@@ -0,0 +1,219 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.operator;
+
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hudi.HudiFlinkStreamer;
+import org.apache.hudi.client.FlinkTaskContextSupplier;
+import org.apache.hudi.client.HoodieFlinkWriteClient;
+import org.apache.hudi.client.common.HoodieFlinkEngineContext;
+import org.apache.hudi.client.common.TaskContextSupplier;
+import org.apache.hudi.common.config.SerializableConfiguration;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieTableType;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.timeline.HoodieTimeline;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.util.StreamerUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+
+/**
+ * Operator helps to generate globally unique instant, it must be executed in one parallelism. Before generate a new
+ * instant , {@link InstantGenerateOperator} will always check whether the last instant has completed. if it is
+ * completed, a new instant will be generated immediately, otherwise, wait and check the state of last instant until
+ * time out and throw an exception.
+ */
+public class InstantGenerateOperator extends AbstractStreamOperator<HoodieRecord> implements OneInputStreamOperator<HoodieRecord, HoodieRecord> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(InstantGenerateOperator.class);
+  public static final String NAME = "InstantGenerateOperator";
+
+  private HudiFlinkStreamer.Config cfg;
+  private HoodieFlinkWriteClient writeClient;
+  private SerializableConfiguration serializableHadoopConf;
+  private transient FileSystem fs;
+  private String latestInstant = "";
+  private List<String> latestInstantList = new ArrayList<>(1);
+  private transient ListState<String> latestInstantState;
+  private List<StreamRecord> bufferedRecords = new LinkedList();
+  private transient ListState<StreamRecord> recordsState;
+  private Integer commitTimeout;
+
+  @Override
+  public void processElement(StreamRecord<HoodieRecord> streamRecord) throws Exception {
+    if (streamRecord.getValue() != null) {
+      bufferedRecords.add(streamRecord);
+      output.collect(streamRecord);
+    }
+  }
+
+  @Override
+  public void open() throws Exception {
+    super.open();
+    // get configs from runtimeContext
+    cfg = (HudiFlinkStreamer.Config) getRuntimeContext().getExecutionConfig().getGlobalJobParameters();
+
+    // timeout
+    commitTimeout = Integer.valueOf(cfg.flinkCommitTimeout);
+
+    // hadoopConf
+    serializableHadoopConf = new SerializableConfiguration(StreamerUtil.getHadoopConf());
+
+    // Hadoop FileSystem
+    fs = FSUtils.getFs(cfg.targetBasePath, serializableHadoopConf.get());
+
+    TaskContextSupplier taskContextSupplier = new FlinkTaskContextSupplier(null);
+
+    // writeClient
+    writeClient = new HoodieFlinkWriteClient(new HoodieFlinkEngineContext(taskContextSupplier), StreamerUtil.getHoodieClientConfig(cfg));
+
+    // init table, create it if not exists.
+    initTable();
+  }
+
+  @Override
+  public void prepareSnapshotPreBarrier(long checkpointId) throws Exception {
+    super.prepareSnapshotPreBarrier(checkpointId);
+    // check whether the last instant is completed, if not, wait 10s and then throws an exception
+    if (!StringUtils.isNullOrEmpty(latestInstant)) {
+      doChecker();
+      // last instant completed, set it empty
+      latestInstant = "";
+    }
+
+    // no data no new instant
+    if (!bufferedRecords.isEmpty()) {
+      latestInstant = startNewInstant(checkpointId);
+    }
+  }
+
+  @Override
+  public void initializeState(StateInitializationContext context) throws Exception {
+    // instantState
+    ListStateDescriptor<String> latestInstantStateDescriptor = new ListStateDescriptor<String>("latestInstant", String.class);
+    latestInstantState = context.getOperatorStateStore().getListState(latestInstantStateDescriptor);
+
+    // recordState
+    ListStateDescriptor<StreamRecord> recordsStateDescriptor = new ListStateDescriptor<StreamRecord>("recordsState", StreamRecord.class);
+    recordsState = context.getOperatorStateStore().getListState(recordsStateDescriptor);
+
+    if (context.isRestored()) {
+      Iterator<String> latestInstantIterator = latestInstantState.get().iterator();
+      latestInstantIterator.forEachRemaining(x -> latestInstant = x);
+      LOG.info("InstantGenerateOperator initializeState get latestInstant [{}]", latestInstant);
+
+      Iterator<StreamRecord> recordIterator = recordsState.get().iterator();
+      bufferedRecords.clear();
+      recordIterator.forEachRemaining(x -> bufferedRecords.add(x));
+    }
+  }
+
+  @Override
+  public void snapshotState(StateSnapshotContext functionSnapshotContext) throws Exception {
+    if (latestInstantList.isEmpty()) {
+      latestInstantList.add(latestInstant);
+    } else {
+      latestInstantList.set(0, latestInstant);
+    }
+    latestInstantState.update(latestInstantList);
+    LOG.info("Update latest instant [{}]", latestInstant);
+
+    recordsState.update(bufferedRecords);
+    LOG.info("Update records state size = [{}]", bufferedRecords.size());
+    bufferedRecords.clear();
+  }
+
+  /**
+   * Create a new instant.
+   *
+   * @param checkpointId
+   */
+  private String startNewInstant(long checkpointId) {
+    String newTime = writeClient.startCommit();
+    LOG.info("create instant [{}], at checkpoint [{}]", newTime, checkpointId);
+    return newTime;
+  }
+
+  /**
+   * Check the status of last instant.
+   */
+  private void doChecker() throws InterruptedException {
+    // query the requested and inflight commit/deltacommit instants
+    String commitType = cfg.tableType.equals(HoodieTableType.COPY_ON_WRITE.name()) ? HoodieTimeline.COMMIT_ACTION : HoodieTimeline.DELTA_COMMIT_ACTION;
+    LOG.info("Query latest instant [{}]", latestInstant);
+    List<String> rollbackPendingCommits = writeClient.getInflightsAndRequestedInstants(commitType);
+    int tryTimes = 0;
+    while (tryTimes < commitTimeout) {

Review comment:
       It seems that this variable: `commitTimeout ` means check retry number? If you think it a timeout solution, then it is not exact counting.

##########
File path: hudi-flink-writer/src/main/java/org/apache/hudi/operator/InstantGenerateOperator.java
##########
@@ -0,0 +1,219 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.operator;
+
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hudi.HudiFlinkStreamer;
+import org.apache.hudi.client.FlinkTaskContextSupplier;
+import org.apache.hudi.client.HoodieFlinkWriteClient;
+import org.apache.hudi.client.common.HoodieFlinkEngineContext;
+import org.apache.hudi.client.common.TaskContextSupplier;
+import org.apache.hudi.common.config.SerializableConfiguration;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieTableType;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.timeline.HoodieTimeline;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.util.StreamerUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+
+/**
+ * Operator helps to generate globally unique instant, it must be executed in one parallelism. Before generate a new
+ * instant , {@link InstantGenerateOperator} will always check whether the last instant has completed. if it is
+ * completed, a new instant will be generated immediately, otherwise, wait and check the state of last instant until
+ * time out and throw an exception.
+ */
+public class InstantGenerateOperator extends AbstractStreamOperator<HoodieRecord> implements OneInputStreamOperator<HoodieRecord, HoodieRecord> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(InstantGenerateOperator.class);
+  public static final String NAME = "InstantGenerateOperator";
+
+  private HudiFlinkStreamer.Config cfg;
+  private HoodieFlinkWriteClient writeClient;
+  private SerializableConfiguration serializableHadoopConf;
+  private transient FileSystem fs;
+  private String latestInstant = "";
+  private List<String> latestInstantList = new ArrayList<>(1);
+  private transient ListState<String> latestInstantState;
+  private List<StreamRecord> bufferedRecords = new LinkedList();
+  private transient ListState<StreamRecord> recordsState;
+  private Integer commitTimeout;
+
+  @Override
+  public void processElement(StreamRecord<HoodieRecord> streamRecord) throws Exception {
+    if (streamRecord.getValue() != null) {
+      bufferedRecords.add(streamRecord);
+      output.collect(streamRecord);
+    }
+  }
+
+  @Override
+  public void open() throws Exception {
+    super.open();
+    // get configs from runtimeContext
+    cfg = (HudiFlinkStreamer.Config) getRuntimeContext().getExecutionConfig().getGlobalJobParameters();
+
+    // timeout
+    commitTimeout = Integer.valueOf(cfg.flinkCommitTimeout);
+
+    // hadoopConf
+    serializableHadoopConf = new SerializableConfiguration(StreamerUtil.getHadoopConf());
+
+    // Hadoop FileSystem
+    fs = FSUtils.getFs(cfg.targetBasePath, serializableHadoopConf.get());
+
+    TaskContextSupplier taskContextSupplier = new FlinkTaskContextSupplier(null);
+
+    // writeClient
+    writeClient = new HoodieFlinkWriteClient(new HoodieFlinkEngineContext(taskContextSupplier), StreamerUtil.getHoodieClientConfig(cfg));
+
+    // init table, create it if not exists.
+    initTable();
+  }
+
+  @Override
+  public void prepareSnapshotPreBarrier(long checkpointId) throws Exception {
+    super.prepareSnapshotPreBarrier(checkpointId);
+    // check whether the last instant is completed, if not, wait 10s and then throws an exception
+    if (!StringUtils.isNullOrEmpty(latestInstant)) {
+      doChecker();
+      // last instant completed, set it empty
+      latestInstant = "";
+    }
+
+    // no data no new instant
+    if (!bufferedRecords.isEmpty()) {
+      latestInstant = startNewInstant(checkpointId);
+    }
+  }
+
+  @Override
+  public void initializeState(StateInitializationContext context) throws Exception {
+    // instantState
+    ListStateDescriptor<String> latestInstantStateDescriptor = new ListStateDescriptor<String>("latestInstant", String.class);
+    latestInstantState = context.getOperatorStateStore().getListState(latestInstantStateDescriptor);
+
+    // recordState
+    ListStateDescriptor<StreamRecord> recordsStateDescriptor = new ListStateDescriptor<StreamRecord>("recordsState", StreamRecord.class);
+    recordsState = context.getOperatorStateStore().getListState(recordsStateDescriptor);
+
+    if (context.isRestored()) {
+      Iterator<String> latestInstantIterator = latestInstantState.get().iterator();
+      latestInstantIterator.forEachRemaining(x -> latestInstant = x);
+      LOG.info("InstantGenerateOperator initializeState get latestInstant [{}]", latestInstant);
+
+      Iterator<StreamRecord> recordIterator = recordsState.get().iterator();
+      bufferedRecords.clear();
+      recordIterator.forEachRemaining(x -> bufferedRecords.add(x));
+    }
+  }
+
+  @Override
+  public void snapshotState(StateSnapshotContext functionSnapshotContext) throws Exception {
+    if (latestInstantList.isEmpty()) {
+      latestInstantList.add(latestInstant);
+    } else {
+      latestInstantList.set(0, latestInstant);
+    }
+    latestInstantState.update(latestInstantList);
+    LOG.info("Update latest instant [{}]", latestInstant);
+
+    recordsState.update(bufferedRecords);
+    LOG.info("Update records state size = [{}]", bufferedRecords.size());
+    bufferedRecords.clear();
+  }
+
+  /**
+   * Create a new instant.
+   *
+   * @param checkpointId
+   */
+  private String startNewInstant(long checkpointId) {
+    String newTime = writeClient.startCommit();
+    LOG.info("create instant [{}], at checkpoint [{}]", newTime, checkpointId);
+    return newTime;
+  }
+
+  /**
+   * Check the status of last instant.
+   */
+  private void doChecker() throws InterruptedException {
+    // query the requested and inflight commit/deltacommit instants
+    String commitType = cfg.tableType.equals(HoodieTableType.COPY_ON_WRITE.name()) ? HoodieTimeline.COMMIT_ACTION : HoodieTimeline.DELTA_COMMIT_ACTION;
+    LOG.info("Query latest instant [{}]", latestInstant);
+    List<String> rollbackPendingCommits = writeClient.getInflightsAndRequestedInstants(commitType);
+    int tryTimes = 0;
+    while (tryTimes < commitTimeout) {
+      tryTimes++;
+      StringBuffer sb = new StringBuffer();
+      if (rollbackPendingCommits.contains(latestInstant)) {
+        //清空 sb

Review comment:
       Chinese? Why this is a clear action to string buffer?

##########
File path: hudi-flink-writer/src/main/java/org/apache/hudi/operator/InstantGenerateOperator.java
##########
@@ -0,0 +1,219 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.operator;
+
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hudi.HudiFlinkStreamer;
+import org.apache.hudi.client.FlinkTaskContextSupplier;
+import org.apache.hudi.client.HoodieFlinkWriteClient;
+import org.apache.hudi.client.common.HoodieFlinkEngineContext;
+import org.apache.hudi.client.common.TaskContextSupplier;
+import org.apache.hudi.common.config.SerializableConfiguration;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieTableType;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.timeline.HoodieTimeline;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.util.StreamerUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+
+/**
+ * Operator helps to generate globally unique instant, it must be executed in one parallelism. Before generate a new
+ * instant , {@link InstantGenerateOperator} will always check whether the last instant has completed. if it is
+ * completed, a new instant will be generated immediately, otherwise, wait and check the state of last instant until
+ * time out and throw an exception.
+ */
+public class InstantGenerateOperator extends AbstractStreamOperator<HoodieRecord> implements OneInputStreamOperator<HoodieRecord, HoodieRecord> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(InstantGenerateOperator.class);
+  public static final String NAME = "InstantGenerateOperator";
+
+  private HudiFlinkStreamer.Config cfg;
+  private HoodieFlinkWriteClient writeClient;
+  private SerializableConfiguration serializableHadoopConf;
+  private transient FileSystem fs;
+  private String latestInstant = "";
+  private List<String> latestInstantList = new ArrayList<>(1);
+  private transient ListState<String> latestInstantState;
+  private List<StreamRecord> bufferedRecords = new LinkedList();
+  private transient ListState<StreamRecord> recordsState;
+  private Integer commitTimeout;
+
+  @Override
+  public void processElement(StreamRecord<HoodieRecord> streamRecord) throws Exception {
+    if (streamRecord.getValue() != null) {
+      bufferedRecords.add(streamRecord);
+      output.collect(streamRecord);
+    }
+  }
+
+  @Override
+  public void open() throws Exception {
+    super.open();
+    // get configs from runtimeContext
+    cfg = (HudiFlinkStreamer.Config) getRuntimeContext().getExecutionConfig().getGlobalJobParameters();
+
+    // timeout
+    commitTimeout = Integer.valueOf(cfg.flinkCommitTimeout);
+
+    // hadoopConf
+    serializableHadoopConf = new SerializableConfiguration(StreamerUtil.getHadoopConf());
+
+    // Hadoop FileSystem
+    fs = FSUtils.getFs(cfg.targetBasePath, serializableHadoopConf.get());
+
+    TaskContextSupplier taskContextSupplier = new FlinkTaskContextSupplier(null);
+
+    // writeClient
+    writeClient = new HoodieFlinkWriteClient(new HoodieFlinkEngineContext(taskContextSupplier), StreamerUtil.getHoodieClientConfig(cfg));
+
+    // init table, create it if not exists.
+    initTable();
+  }
+
+  @Override
+  public void prepareSnapshotPreBarrier(long checkpointId) throws Exception {
+    super.prepareSnapshotPreBarrier(checkpointId);
+    // check whether the last instant is completed, if not, wait 10s and then throws an exception
+    if (!StringUtils.isNullOrEmpty(latestInstant)) {
+      doChecker();
+      // last instant completed, set it empty
+      latestInstant = "";
+    }
+
+    // no data no new instant
+    if (!bufferedRecords.isEmpty()) {
+      latestInstant = startNewInstant(checkpointId);
+    }
+  }
+
+  @Override
+  public void initializeState(StateInitializationContext context) throws Exception {
+    // instantState
+    ListStateDescriptor<String> latestInstantStateDescriptor = new ListStateDescriptor<String>("latestInstant", String.class);
+    latestInstantState = context.getOperatorStateStore().getListState(latestInstantStateDescriptor);
+
+    // recordState
+    ListStateDescriptor<StreamRecord> recordsStateDescriptor = new ListStateDescriptor<StreamRecord>("recordsState", StreamRecord.class);
+    recordsState = context.getOperatorStateStore().getListState(recordsStateDescriptor);
+
+    if (context.isRestored()) {
+      Iterator<String> latestInstantIterator = latestInstantState.get().iterator();
+      latestInstantIterator.forEachRemaining(x -> latestInstant = x);
+      LOG.info("InstantGenerateOperator initializeState get latestInstant [{}]", latestInstant);
+
+      Iterator<StreamRecord> recordIterator = recordsState.get().iterator();
+      bufferedRecords.clear();
+      recordIterator.forEachRemaining(x -> bufferedRecords.add(x));
+    }
+  }
+
+  @Override
+  public void snapshotState(StateSnapshotContext functionSnapshotContext) throws Exception {
+    if (latestInstantList.isEmpty()) {
+      latestInstantList.add(latestInstant);
+    } else {
+      latestInstantList.set(0, latestInstant);
+    }
+    latestInstantState.update(latestInstantList);
+    LOG.info("Update latest instant [{}]", latestInstant);
+
+    recordsState.update(bufferedRecords);
+    LOG.info("Update records state size = [{}]", bufferedRecords.size());
+    bufferedRecords.clear();
+  }
+
+  /**
+   * Create a new instant.
+   *
+   * @param checkpointId
+   */
+  private String startNewInstant(long checkpointId) {
+    String newTime = writeClient.startCommit();
+    LOG.info("create instant [{}], at checkpoint [{}]", newTime, checkpointId);
+    return newTime;
+  }
+
+  /**
+   * Check the status of last instant.
+   */
+  private void doChecker() throws InterruptedException {

Review comment:
       IMO, `checker` is a noun. Can we name it to be `doCheck` or `checkXXX `?

##########
File path: hudi-flink-writer/src/main/java/org/apache/hudi/operator/InstantGenerateOperator.java
##########
@@ -0,0 +1,219 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.operator;
+
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hudi.HudiFlinkStreamer;
+import org.apache.hudi.client.FlinkTaskContextSupplier;
+import org.apache.hudi.client.HoodieFlinkWriteClient;
+import org.apache.hudi.client.common.HoodieFlinkEngineContext;
+import org.apache.hudi.client.common.TaskContextSupplier;
+import org.apache.hudi.common.config.SerializableConfiguration;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieTableType;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.timeline.HoodieTimeline;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.util.StreamerUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+
+/**
+ * Operator helps to generate globally unique instant, it must be executed in one parallelism. Before generate a new
+ * instant , {@link InstantGenerateOperator} will always check whether the last instant has completed. if it is
+ * completed, a new instant will be generated immediately, otherwise, wait and check the state of last instant until
+ * time out and throw an exception.
+ */
+public class InstantGenerateOperator extends AbstractStreamOperator<HoodieRecord> implements OneInputStreamOperator<HoodieRecord, HoodieRecord> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(InstantGenerateOperator.class);
+  public static final String NAME = "InstantGenerateOperator";
+
+  private HudiFlinkStreamer.Config cfg;
+  private HoodieFlinkWriteClient writeClient;
+  private SerializableConfiguration serializableHadoopConf;
+  private transient FileSystem fs;
+  private String latestInstant = "";
+  private List<String> latestInstantList = new ArrayList<>(1);
+  private transient ListState<String> latestInstantState;
+  private List<StreamRecord> bufferedRecords = new LinkedList();
+  private transient ListState<StreamRecord> recordsState;
+  private Integer commitTimeout;
+
+  @Override
+  public void processElement(StreamRecord<HoodieRecord> streamRecord) throws Exception {
+    if (streamRecord.getValue() != null) {
+      bufferedRecords.add(streamRecord);
+      output.collect(streamRecord);
+    }
+  }
+
+  @Override
+  public void open() throws Exception {
+    super.open();
+    // get configs from runtimeContext
+    cfg = (HudiFlinkStreamer.Config) getRuntimeContext().getExecutionConfig().getGlobalJobParameters();
+
+    // timeout
+    commitTimeout = Integer.valueOf(cfg.flinkCommitTimeout);
+
+    // hadoopConf
+    serializableHadoopConf = new SerializableConfiguration(StreamerUtil.getHadoopConf());
+
+    // Hadoop FileSystem
+    fs = FSUtils.getFs(cfg.targetBasePath, serializableHadoopConf.get());
+
+    TaskContextSupplier taskContextSupplier = new FlinkTaskContextSupplier(null);
+
+    // writeClient
+    writeClient = new HoodieFlinkWriteClient(new HoodieFlinkEngineContext(taskContextSupplier), StreamerUtil.getHoodieClientConfig(cfg));
+
+    // init table, create it if not exists.
+    initTable();
+  }
+
+  @Override
+  public void prepareSnapshotPreBarrier(long checkpointId) throws Exception {
+    super.prepareSnapshotPreBarrier(checkpointId);
+    // check whether the last instant is completed, if not, wait 10s and then throws an exception
+    if (!StringUtils.isNullOrEmpty(latestInstant)) {
+      doChecker();
+      // last instant completed, set it empty
+      latestInstant = "";
+    }
+
+    // no data no new instant
+    if (!bufferedRecords.isEmpty()) {
+      latestInstant = startNewInstant(checkpointId);
+    }
+  }
+
+  @Override
+  public void initializeState(StateInitializationContext context) throws Exception {
+    // instantState
+    ListStateDescriptor<String> latestInstantStateDescriptor = new ListStateDescriptor<String>("latestInstant", String.class);
+    latestInstantState = context.getOperatorStateStore().getListState(latestInstantStateDescriptor);
+
+    // recordState
+    ListStateDescriptor<StreamRecord> recordsStateDescriptor = new ListStateDescriptor<StreamRecord>("recordsState", StreamRecord.class);
+    recordsState = context.getOperatorStateStore().getListState(recordsStateDescriptor);
+
+    if (context.isRestored()) {
+      Iterator<String> latestInstantIterator = latestInstantState.get().iterator();
+      latestInstantIterator.forEachRemaining(x -> latestInstant = x);
+      LOG.info("InstantGenerateOperator initializeState get latestInstant [{}]", latestInstant);
+
+      Iterator<StreamRecord> recordIterator = recordsState.get().iterator();
+      bufferedRecords.clear();
+      recordIterator.forEachRemaining(x -> bufferedRecords.add(x));
+    }
+  }
+
+  @Override
+  public void snapshotState(StateSnapshotContext functionSnapshotContext) throws Exception {
+    if (latestInstantList.isEmpty()) {
+      latestInstantList.add(latestInstant);
+    } else {
+      latestInstantList.set(0, latestInstant);
+    }
+    latestInstantState.update(latestInstantList);
+    LOG.info("Update latest instant [{}]", latestInstant);
+
+    recordsState.update(bufferedRecords);
+    LOG.info("Update records state size = [{}]", bufferedRecords.size());
+    bufferedRecords.clear();
+  }
+
+  /**
+   * Create a new instant.
+   *
+   * @param checkpointId
+   */
+  private String startNewInstant(long checkpointId) {
+    String newTime = writeClient.startCommit();
+    LOG.info("create instant [{}], at checkpoint [{}]", newTime, checkpointId);
+    return newTime;
+  }
+
+  /**
+   * Check the status of last instant.
+   */
+  private void doChecker() throws InterruptedException {
+    // query the requested and inflight commit/deltacommit instants
+    String commitType = cfg.tableType.equals(HoodieTableType.COPY_ON_WRITE.name()) ? HoodieTimeline.COMMIT_ACTION : HoodieTimeline.DELTA_COMMIT_ACTION;
+    LOG.info("Query latest instant [{}]", latestInstant);
+    List<String> rollbackPendingCommits = writeClient.getInflightsAndRequestedInstants(commitType);
+    int tryTimes = 0;
+    while (tryTimes < commitTimeout) {

Review comment:
       IMO, we can introduce both `retry number` and `timeout` conditions. WDYT?

##########
File path: hudi-flink-writer/src/main/java/org/apache/hudi/operator/InstantGenerateOperator.java
##########
@@ -0,0 +1,219 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.operator;
+
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hudi.HudiFlinkStreamer;
+import org.apache.hudi.client.FlinkTaskContextSupplier;
+import org.apache.hudi.client.HoodieFlinkWriteClient;
+import org.apache.hudi.client.common.HoodieFlinkEngineContext;
+import org.apache.hudi.client.common.TaskContextSupplier;
+import org.apache.hudi.common.config.SerializableConfiguration;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieTableType;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.timeline.HoodieTimeline;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.util.StreamerUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+
+/**
+ * Operator helps to generate globally unique instant, it must be executed in one parallelism. Before generate a new
+ * instant , {@link InstantGenerateOperator} will always check whether the last instant has completed. if it is
+ * completed, a new instant will be generated immediately, otherwise, wait and check the state of last instant until
+ * time out and throw an exception.
+ */
+public class InstantGenerateOperator extends AbstractStreamOperator<HoodieRecord> implements OneInputStreamOperator<HoodieRecord, HoodieRecord> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(InstantGenerateOperator.class);
+  public static final String NAME = "InstantGenerateOperator";
+
+  private HudiFlinkStreamer.Config cfg;
+  private HoodieFlinkWriteClient writeClient;
+  private SerializableConfiguration serializableHadoopConf;
+  private transient FileSystem fs;
+  private String latestInstant = "";
+  private List<String> latestInstantList = new ArrayList<>(1);
+  private transient ListState<String> latestInstantState;
+  private List<StreamRecord> bufferedRecords = new LinkedList();
+  private transient ListState<StreamRecord> recordsState;
+  private Integer commitTimeout;
+
+  @Override
+  public void processElement(StreamRecord<HoodieRecord> streamRecord) throws Exception {
+    if (streamRecord.getValue() != null) {
+      bufferedRecords.add(streamRecord);
+      output.collect(streamRecord);
+    }
+  }
+
+  @Override
+  public void open() throws Exception {
+    super.open();
+    // get configs from runtimeContext
+    cfg = (HudiFlinkStreamer.Config) getRuntimeContext().getExecutionConfig().getGlobalJobParameters();
+
+    // timeout
+    commitTimeout = Integer.valueOf(cfg.flinkCommitTimeout);
+
+    // hadoopConf
+    serializableHadoopConf = new SerializableConfiguration(StreamerUtil.getHadoopConf());
+
+    // Hadoop FileSystem
+    fs = FSUtils.getFs(cfg.targetBasePath, serializableHadoopConf.get());
+
+    TaskContextSupplier taskContextSupplier = new FlinkTaskContextSupplier(null);
+
+    // writeClient
+    writeClient = new HoodieFlinkWriteClient(new HoodieFlinkEngineContext(taskContextSupplier), StreamerUtil.getHoodieClientConfig(cfg));
+
+    // init table, create it if not exists.
+    initTable();
+  }
+
+  @Override
+  public void prepareSnapshotPreBarrier(long checkpointId) throws Exception {
+    super.prepareSnapshotPreBarrier(checkpointId);
+    // check whether the last instant is completed, if not, wait 10s and then throws an exception
+    if (!StringUtils.isNullOrEmpty(latestInstant)) {
+      doChecker();
+      // last instant completed, set it empty
+      latestInstant = "";
+    }
+
+    // no data no new instant
+    if (!bufferedRecords.isEmpty()) {
+      latestInstant = startNewInstant(checkpointId);
+    }
+  }
+
+  @Override
+  public void initializeState(StateInitializationContext context) throws Exception {
+    // instantState
+    ListStateDescriptor<String> latestInstantStateDescriptor = new ListStateDescriptor<String>("latestInstant", String.class);
+    latestInstantState = context.getOperatorStateStore().getListState(latestInstantStateDescriptor);
+
+    // recordState
+    ListStateDescriptor<StreamRecord> recordsStateDescriptor = new ListStateDescriptor<StreamRecord>("recordsState", StreamRecord.class);
+    recordsState = context.getOperatorStateStore().getListState(recordsStateDescriptor);
+
+    if (context.isRestored()) {
+      Iterator<String> latestInstantIterator = latestInstantState.get().iterator();
+      latestInstantIterator.forEachRemaining(x -> latestInstant = x);
+      LOG.info("InstantGenerateOperator initializeState get latestInstant [{}]", latestInstant);
+
+      Iterator<StreamRecord> recordIterator = recordsState.get().iterator();
+      bufferedRecords.clear();
+      recordIterator.forEachRemaining(x -> bufferedRecords.add(x));
+    }
+  }
+
+  @Override
+  public void snapshotState(StateSnapshotContext functionSnapshotContext) throws Exception {
+    if (latestInstantList.isEmpty()) {
+      latestInstantList.add(latestInstant);
+    } else {
+      latestInstantList.set(0, latestInstant);
+    }
+    latestInstantState.update(latestInstantList);
+    LOG.info("Update latest instant [{}]", latestInstant);
+
+    recordsState.update(bufferedRecords);
+    LOG.info("Update records state size = [{}]", bufferedRecords.size());
+    bufferedRecords.clear();
+  }
+
+  /**
+   * Create a new instant.
+   *
+   * @param checkpointId
+   */
+  private String startNewInstant(long checkpointId) {
+    String newTime = writeClient.startCommit();
+    LOG.info("create instant [{}], at checkpoint [{}]", newTime, checkpointId);
+    return newTime;
+  }
+
+  /**
+   * Check the status of last instant.
+   */
+  private void doChecker() throws InterruptedException {
+    // query the requested and inflight commit/deltacommit instants
+    String commitType = cfg.tableType.equals(HoodieTableType.COPY_ON_WRITE.name()) ? HoodieTimeline.COMMIT_ACTION : HoodieTimeline.DELTA_COMMIT_ACTION;
+    LOG.info("Query latest instant [{}]", latestInstant);
+    List<String> rollbackPendingCommits = writeClient.getInflightsAndRequestedInstants(commitType);
+    int tryTimes = 0;
+    while (tryTimes < commitTimeout) {
+      tryTimes++;
+      StringBuffer sb = new StringBuffer();
+      if (rollbackPendingCommits.contains(latestInstant)) {
+        //清空 sb
+        rollbackPendingCommits.forEach(x -> sb.append(x).append(","));
+
+        LOG.warn("Latest transaction [{}] is not completed! unCompleted transaction:[{}],try times [{}]", latestInstant, sb.toString(), tryTimes);
+
+        Thread.sleep(1000);
+        rollbackPendingCommits = writeClient.getInflightsAndRequestedInstants(commitType);
+      } else {
+        LOG.warn("Latest transaction [{}] is completed! Completed transaction, try times [{}]", latestInstant, tryTimes);
+        return;
+      }
+    }
+    throw new InterruptedException("Last instant costs more than ten second, stop task now");
+  }
+
+
+  /**
+   * Create table if not exists.
+   */
+  private void initTable() throws IOException {
+    if (!fs.exists(new Path(cfg.targetBasePath))) {

Review comment:
       What's the result if the path exists? Adding a log warning message?




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