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 2022/01/24 12:02:26 UTC

[GitHub] [hudi] garyli1019 opened a new pull request #4679: [RFC-35] Make Flink writer stream friendly

garyli1019 opened a new pull request #4679:
URL: https://github.com/apache/hudi/pull/4679


   ## *Tips*
   - *Thank you very much for contributing to Apache Hudi.*
   - *Please review https://hudi.apache.org/contribute/how-to-contribute before opening a pull request.*
   
   ## What is the purpose of the pull request
   
   *(For example: This pull request adds quick-start document.)*
   
   ## Brief change log
   
   *(for example:)*
     - *Modify AnnotationLocation checkstyle rule in checkstyle.xml*
   
   ## Verify this pull request
   
   *(Please pick either of the following options)*
   
   This pull request is a trivial rework / code cleanup without any test coverage.
   
   *(or)*
   
   This pull request is already covered by existing tests, such as *(please describe tests)*.
   
   (or)
   
   This change added tests and can be verified as follows:
   
   *(example:)*
   
     - *Added integration tests for end-to-end.*
     - *Added HoodieClientWriteTest to verify the change.*
     - *Manually verified the change by running a job locally.*
   
   ## Committer checklist
   
    - [ ] Has a corresponding JIRA in PR title & commit
    
    - [ ] Commit message is descriptive of the change
    
    - [ ] CI is green
   
    - [ ] Necessary doc changes done or have another open PR
          
    - [ ] For large changes, please consider breaking it into sub-tasks under an umbrella JIRA.
   


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] garyli1019 commented on a change in pull request #4679: [HUDI-3315] RFC-35 Part-1 Support bucket index in Flink writer

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



##########
File path: hudi-flink/src/main/java/org/apache/hudi/sink/BucketStreamWriteFunction.java
##########
@@ -0,0 +1,158 @@
+/*
+ * 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.configuration.Configuration;
+import org.apache.flink.runtime.state.FunctionInitializationContext;
+import org.apache.flink.streaming.api.functions.ProcessFunction;
+import org.apache.flink.util.Collector;
+import org.apache.hudi.common.model.FileSlice;
+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.table.timeline.HoodieInstant;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.configuration.FlinkOptions;
+import org.apache.hudi.index.bucket.BucketIdentifier;
+import org.apache.hudi.table.HoodieFlinkTable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+
+import static java.util.stream.Collectors.toList;
+
+public class BucketStreamWriteFunction<I> extends StreamWriteFunction<I> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BucketStreamWriteFunction.class);
+
+  private int maxParallelism;
+
+  private int parallelism;
+
+  private int hiveBucketNum;
+
+  protected transient HoodieFlinkTable table;
+
+  private String indexKeyFields;
+
+  private HashMap<String, String> bucketToFileIDMap;
+
+  /**
+   * Constructs a BucketStreamWriteFunction.
+   *
+   * @param config The config options
+   */
+  public BucketStreamWriteFunction(Configuration config) {
+    super(config);
+    this.bucketToFileIDMap = new HashMap<String, String>();
+  }
+
+  @Override
+  public void open(Configuration parameters) throws IOException {
+    super.open(parameters);
+    this.hiveBucketNum = config.getInteger(FlinkOptions.BUCKET_INDEX_NUM_BUCKETS);
+    this.indexKeyFields = config.getString(FlinkOptions.INDEX_KEY_FIELD);
+    this.taskID = getRuntimeContext().getIndexOfThisSubtask();
+    this.parallelism = getRuntimeContext().getNumberOfParallelSubtasks();
+    this.maxParallelism = getRuntimeContext().getMaxNumberOfParallelSubtasks();
+    bootstrapIndex();
+  }
+
+  @Override
+  public void initializeState(FunctionInitializationContext context) throws Exception {
+    super.initializeState(context);
+    this.table = this.writeClient.getHoodieTable();
+  }
+
+  @Override
+  public void processElement(I i, ProcessFunction<I, Object>.Context context, Collector<Object> collector) throws Exception {
+    HoodieRecord<?> record = (HoodieRecord<?>) i;
+    final HoodieKey hoodieKey = record.getKey();
+    final HoodieRecordLocation location;
+
+    final int bucketNum = BucketIdentifier.getBucketId(hoodieKey, indexKeyFields, hiveBucketNum);
+    final String partitionBucketId = BucketIdentifier.partitionBucketIdStr(hoodieKey.getPartitionPath(), bucketNum);
+
+    if (bucketToFileIDMap.containsKey(partitionBucketId)) {
+      location = new HoodieRecordLocation("U", bucketToFileIDMap.get(partitionBucketId));
+    } else {
+      String newFileId = BucketIdentifier.newBucketFileIdPrefix(bucketNum);
+      location = new HoodieRecordLocation("I", newFileId);
+      bucketToFileIDMap.put(partitionBucketId, newFileId);
+    }
+    record.unseal();
+    record.setCurrentLocation(location);
+    record.seal();
+    bufferRecord(record);
+  }
+
+  /**
+   * Get partition_bucket -> fileID mapping from the existing hudi table.
+   * This is a required operation for each restart to avoid having duplicate file ids for one bucket.
+   */
+  private void bootstrapIndex() throws IOException {
+    Option<HoodieInstant> latestCommitTime = table.getFileSystemView().getTimeline().filterCompletedInstants().lastInstant();
+    if (!latestCommitTime.isPresent()) {
+      return;
+    }
+    // bootstrap bucket info from existing file system
+    // bucketNum % totalParallelism == this taskID belongs to this task
+    HashSet<Integer> bucketToLoad = new HashSet<>();
+    for (int i = 0; i < hiveBucketNum; i++) {
+      int partitionOfBucket = BucketIdentifier.mod(i, parallelism);

Review comment:
       removed the negative condition check but keep this mod method for readability. 




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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] danny0405 commented on pull request #4679: [HUDI-3315] RFC-35 Part-1 Support bucket index in Flink writer

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


   > @wxplovecc hi, this issue looks like not related to this PR, would you submit an issue then we can take a look.
   
   Seems a bug 🐛 of the 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.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] danny0405 commented on a change in pull request #4679: [HUDI-3315] RFC-35 Part-1 Support bucket index in Flink writer

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



##########
File path: hudi-flink/src/main/java/org/apache/hudi/sink/utils/Pipelines.java
##########
@@ -189,21 +191,32 @@
   }
 
   public static DataStream<Object> hoodieStreamWrite(Configuration conf, int defaultParallelism, DataStream<HoodieRecord> dataStream) {
-    WriteOperatorFactory<HoodieRecord> operatorFactory = StreamWriteOperator.getFactory(conf);
-    return dataStream

Review comment:
       Yeah, if we can make both the code in `Pipelines` and `HoodieTableSink` clean ~




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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] garyli1019 commented on a change in pull request #4679: [HUDI-3315] RFC-35 Part-1 Support bucket index in Flink writer

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



##########
File path: hudi-flink/src/main/java/org/apache/hudi/sink/BucketStreamWriteFunction.java
##########
@@ -0,0 +1,158 @@
+/*
+ * 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.configuration.Configuration;
+import org.apache.flink.runtime.state.FunctionInitializationContext;
+import org.apache.flink.streaming.api.functions.ProcessFunction;
+import org.apache.flink.util.Collector;
+import org.apache.hudi.common.model.FileSlice;
+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.table.timeline.HoodieInstant;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.configuration.FlinkOptions;
+import org.apache.hudi.index.bucket.BucketIdentifier;
+import org.apache.hudi.table.HoodieFlinkTable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+
+import static java.util.stream.Collectors.toList;
+
+public class BucketStreamWriteFunction<I> extends StreamWriteFunction<I> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BucketStreamWriteFunction.class);
+
+  private int maxParallelism;
+
+  private int parallelism;
+
+  private int hiveBucketNum;
+
+  protected transient HoodieFlinkTable table;
+
+  private String indexKeyFields;
+
+  private HashMap<String, String> bucketToFileIDMap;
+
+  /**
+   * Constructs a BucketStreamWriteFunction.
+   *
+   * @param config The config options
+   */
+  public BucketStreamWriteFunction(Configuration config) {
+    super(config);
+    this.bucketToFileIDMap = new HashMap<String, String>();
+  }
+
+  @Override
+  public void open(Configuration parameters) throws IOException {
+    super.open(parameters);
+    this.hiveBucketNum = config.getInteger(FlinkOptions.BUCKET_INDEX_NUM_BUCKETS);
+    this.indexKeyFields = config.getString(FlinkOptions.INDEX_KEY_FIELD);
+    this.taskID = getRuntimeContext().getIndexOfThisSubtask();
+    this.parallelism = getRuntimeContext().getNumberOfParallelSubtasks();
+    this.maxParallelism = getRuntimeContext().getMaxNumberOfParallelSubtasks();
+    bootstrapIndex();
+  }
+
+  @Override
+  public void initializeState(FunctionInitializationContext context) throws Exception {
+    super.initializeState(context);
+    this.table = this.writeClient.getHoodieTable();
+  }
+
+  @Override
+  public void processElement(I i, ProcessFunction<I, Object>.Context context, Collector<Object> collector) throws Exception {
+    HoodieRecord<?> record = (HoodieRecord<?>) i;
+    final HoodieKey hoodieKey = record.getKey();
+    final HoodieRecordLocation location;
+
+    final int bucketNum = BucketIdentifier.getBucketId(hoodieKey, indexKeyFields, hiveBucketNum);
+    final String partitionBucketId = BucketIdentifier.partitionBucketIdStr(hoodieKey.getPartitionPath(), bucketNum);
+
+    if (bucketToFileIDMap.containsKey(partitionBucketId)) {
+      location = new HoodieRecordLocation("U", bucketToFileIDMap.get(partitionBucketId));
+    } else {
+      String newFileId = BucketIdentifier.newBucketFileIdPrefix(bucketNum);
+      location = new HoodieRecordLocation("I", newFileId);
+      bucketToFileIDMap.put(partitionBucketId, newFileId);
+    }
+    record.unseal();
+    record.setCurrentLocation(location);
+    record.seal();
+    bufferRecord(record);
+  }
+
+  /**
+   * Get partition_bucket -> fileID mapping from the existing hudi table.
+   * This is a required operation for each restart to avoid having duplicate file ids for one bucket.
+   */
+  private void bootstrapIndex() throws IOException {
+    Option<HoodieInstant> latestCommitTime = table.getFileSystemView().getTimeline().filterCompletedInstants().lastInstant();
+    if (!latestCommitTime.isPresent()) {
+      return;
+    }
+    // bootstrap bucket info from existing file system
+    // bucketNum % totalParallelism == this taskID belongs to this task
+    HashSet<Integer> bucketToLoad = new HashSet<>();
+    for (int i = 0; i < hiveBucketNum; i++) {
+      int partitionOfBucket = BucketIdentifier.mod(i, parallelism);
+      if (partitionOfBucket == taskID) {
+        LOG.info(String.format("Bootstrapping index. Adding bucket %s , "
+            + "Current parallelism: %s , Max parallelism: %s , Current task id: %s",
+            i, parallelism, maxParallelism, taskID));
+        bucketToLoad.add(i);
+      }
+    }
+    bucketToLoad.stream().forEach(bucket -> LOG.info(String.format("bucketToLoad contains %s", bucket)));
+
+    LOG.info(String.format("Loading Hoodie Table %s, with path %s", table.getMetaClient().getTableConfig().getTableName(),
+        table.getMetaClient().getBasePath()));
+
+    // Iterate through all existing partitions to load existing fileID belongs to this task
+    List<String> partitions = table.getMetadata().getAllPartitionPaths();

Review comment:
       https://issues.apache.org/jira/browse/HUDI-3539




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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] loukey-lj commented on a change in pull request #4679: [HUDI-3315] RFC-35 Part-1 Support bucket index in Flink writer

Posted by GitBox <gi...@apache.org>.
loukey-lj commented on a change in pull request #4679:
URL: https://github.com/apache/hudi/pull/4679#discussion_r811710014



##########
File path: hudi-flink/src/main/java/org/apache/hudi/sink/BucketStreamWriteFunction.java
##########
@@ -0,0 +1,158 @@
+/*
+ * 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.configuration.Configuration;
+import org.apache.flink.runtime.state.FunctionInitializationContext;
+import org.apache.flink.streaming.api.functions.ProcessFunction;
+import org.apache.flink.util.Collector;
+import org.apache.hudi.common.model.FileSlice;
+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.table.timeline.HoodieInstant;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.configuration.FlinkOptions;
+import org.apache.hudi.index.bucket.BucketIdentifier;
+import org.apache.hudi.table.HoodieFlinkTable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+
+import static java.util.stream.Collectors.toList;
+
+public class BucketStreamWriteFunction<I> extends StreamWriteFunction<I> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BucketStreamWriteFunction.class);
+
+  private int taskTotalParallelism;
+
+  private int currentParallelism;
+
+  private int hiveBucketNum;
+
+  protected transient HoodieFlinkTable table;
+
+  private String indexKeyFields;
+
+  private HashMap<String, String> bucketToFileIDMap;
+
+  /**
+   * Constructs a BucketStreamWriteFunction.
+   *
+   * @param config The config options
+   */
+  public BucketStreamWriteFunction(Configuration config) {
+    super(config);
+    this.bucketToFileIDMap = new HashMap<String, String>();
+  }
+
+  @Override
+  public void open(Configuration parameters) throws IOException {
+    super.open(parameters);
+    this.hiveBucketNum = config.getInteger(FlinkOptions.BUCKET_INDEX_NUM_BUCKETS);
+    this.indexKeyFields = config.getString(FlinkOptions.INDEX_KEY_FIELD);
+    this.taskID = getRuntimeContext().getIndexOfThisSubtask();
+    this.currentParallelism = getRuntimeContext().getNumberOfParallelSubtasks();
+    this.taskTotalParallelism = getRuntimeContext().getMaxNumberOfParallelSubtasks();
+    bootstrapIndex();
+  }
+
+  @Override
+  public void initializeState(FunctionInitializationContext context) throws Exception {
+    super.initializeState(context);
+    this.table = this.writeClient.getHoodieTable();
+  }
+
+  @Override
+  public void processElement(I i, ProcessFunction<I, Object>.Context context, Collector<Object> collector) throws Exception {
+    HoodieRecord<?> record = (HoodieRecord<?>) i;
+    final HoodieKey hoodieKey = record.getKey();
+    final HoodieRecordLocation location;
+
+    final int bucketNum = BucketIdentifier.getBucketId(hoodieKey, indexKeyFields, hiveBucketNum);
+    final String partitionBucketId = BucketIdentifier.partitionBucketIdStr(hoodieKey.getPartitionPath(), bucketNum);
+
+    if (bucketToFileIDMap.containsKey(partitionBucketId)) {

Review comment:
       Partition changes are not supported?




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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] garyli1019 commented on pull request #4679: [HUDI-3315] RFC-35 Part-1 Support bucket index in Flink writer

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


   @hudi-bot run azure


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] hudi-bot removed a comment on pull request #4679: [HUDI-3315] RFC-35 Part-1 Support bucket index in Flink writer

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5471",
       "triggerID" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5491",
       "triggerID" : "1020748430",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5492",
       "triggerID" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5493",
       "triggerID" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5528",
       "triggerID" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6056",
       "triggerID" : "1041367799",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6056",
       "triggerID" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1041367799",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6189",
       "triggerID" : "1047380901",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6189",
       "triggerID" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1047380901",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6195",
       "triggerID" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6196",
       "triggerID" : "1047494529",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6195",
       "triggerID" : "1047494529",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6271",
       "triggerID" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6272",
       "triggerID" : "1049827248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6271",
       "triggerID" : "1049827248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6299",
       "triggerID" : "1050610982",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * 96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6272) Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6271) Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6299) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] minihippo commented on a change in pull request #4679: [HUDI-3315] RFC-35 Part-1 Support bucket index in Flink writer

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



##########
File path: hudi-flink/src/main/java/org/apache/hudi/sink/BucketStreamWriteFunction.java
##########
@@ -0,0 +1,158 @@
+/*
+ * 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.configuration.Configuration;
+import org.apache.flink.runtime.state.FunctionInitializationContext;
+import org.apache.flink.streaming.api.functions.ProcessFunction;
+import org.apache.flink.util.Collector;
+import org.apache.hudi.common.model.FileSlice;
+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.table.timeline.HoodieInstant;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.configuration.FlinkOptions;
+import org.apache.hudi.index.bucket.BucketIdentifier;
+import org.apache.hudi.table.HoodieFlinkTable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+
+import static java.util.stream.Collectors.toList;
+
+public class BucketStreamWriteFunction<I> extends StreamWriteFunction<I> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BucketStreamWriteFunction.class);
+
+  private int maxParallelism;
+
+  private int parallelism;
+
+  private int hiveBucketNum;
+
+  protected transient HoodieFlinkTable table;
+
+  private String indexKeyFields;
+
+  private HashMap<String, String> bucketToFileIDMap;
+
+  /**
+   * Constructs a BucketStreamWriteFunction.
+   *
+   * @param config The config options
+   */
+  public BucketStreamWriteFunction(Configuration config) {
+    super(config);
+    this.bucketToFileIDMap = new HashMap<String, String>();
+  }
+
+  @Override
+  public void open(Configuration parameters) throws IOException {
+    super.open(parameters);
+    this.hiveBucketNum = config.getInteger(FlinkOptions.BUCKET_INDEX_NUM_BUCKETS);
+    this.indexKeyFields = config.getString(FlinkOptions.INDEX_KEY_FIELD);
+    this.taskID = getRuntimeContext().getIndexOfThisSubtask();
+    this.parallelism = getRuntimeContext().getNumberOfParallelSubtasks();
+    this.maxParallelism = getRuntimeContext().getMaxNumberOfParallelSubtasks();
+    bootstrapIndex();
+  }
+
+  @Override
+  public void initializeState(FunctionInitializationContext context) throws Exception {
+    super.initializeState(context);
+    this.table = this.writeClient.getHoodieTable();
+  }
+
+  @Override
+  public void processElement(I i, ProcessFunction<I, Object>.Context context, Collector<Object> collector) throws Exception {
+    HoodieRecord<?> record = (HoodieRecord<?>) i;
+    final HoodieKey hoodieKey = record.getKey();
+    final HoodieRecordLocation location;
+
+    final int bucketNum = BucketIdentifier.getBucketId(hoodieKey, indexKeyFields, hiveBucketNum);
+    final String partitionBucketId = BucketIdentifier.partitionBucketIdStr(hoodieKey.getPartitionPath(), bucketNum);
+
+    if (bucketToFileIDMap.containsKey(partitionBucketId)) {
+      location = new HoodieRecordLocation("U", bucketToFileIDMap.get(partitionBucketId));
+    } else {
+      String newFileId = BucketIdentifier.newBucketFileIdPrefix(bucketNum);
+      location = new HoodieRecordLocation("I", newFileId);
+      bucketToFileIDMap.put(partitionBucketId, newFileId);
+    }
+    record.unseal();
+    record.setCurrentLocation(location);
+    record.seal();
+    bufferRecord(record);
+  }
+
+  /**
+   * Get partition_bucket -> fileID mapping from the existing hudi table.
+   * This is a required operation for each restart to avoid having duplicate file ids for one bucket.
+   */
+  private void bootstrapIndex() throws IOException {
+    Option<HoodieInstant> latestCommitTime = table.getFileSystemView().getTimeline().filterCompletedInstants().lastInstant();
+    if (!latestCommitTime.isPresent()) {
+      return;
+    }
+    // bootstrap bucket info from existing file system
+    // bucketNum % totalParallelism == this taskID belongs to this task
+    HashSet<Integer> bucketToLoad = new HashSet<>();
+    for (int i = 0; i < hiveBucketNum; i++) {
+      int partitionOfBucket = BucketIdentifier.mod(i, parallelism);

Review comment:
       just change to i%parallelism. Cause `i` can not be negative.




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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] garyli1019 commented on a change in pull request #4679: [HUDI-3315] RFC-35 Part-1 Support bucket index in Flink writer

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



##########
File path: hudi-flink/src/main/java/org/apache/hudi/sink/BucketStreamWriteFunction.java
##########
@@ -0,0 +1,158 @@
+/*
+ * 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.configuration.Configuration;
+import org.apache.flink.runtime.state.FunctionInitializationContext;
+import org.apache.flink.streaming.api.functions.ProcessFunction;
+import org.apache.flink.util.Collector;
+import org.apache.hudi.common.model.FileSlice;
+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.table.timeline.HoodieInstant;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.configuration.FlinkOptions;
+import org.apache.hudi.index.bucket.BucketIdentifier;
+import org.apache.hudi.table.HoodieFlinkTable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+
+import static java.util.stream.Collectors.toList;
+
+public class BucketStreamWriteFunction<I> extends StreamWriteFunction<I> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BucketStreamWriteFunction.class);
+
+  private int taskTotalParallelism;
+
+  private int currentParallelism;
+
+  private int hiveBucketNum;
+
+  protected transient HoodieFlinkTable table;
+
+  private String indexKeyFields;
+
+  private HashMap<String, String> bucketToFileIDMap;
+
+  /**
+   * Constructs a BucketStreamWriteFunction.
+   *
+   * @param config The config options
+   */
+  public BucketStreamWriteFunction(Configuration config) {
+    super(config);
+    this.bucketToFileIDMap = new HashMap<String, String>();
+  }
+
+  @Override
+  public void open(Configuration parameters) throws IOException {
+    super.open(parameters);
+    this.hiveBucketNum = config.getInteger(FlinkOptions.BUCKET_INDEX_NUM_BUCKETS);
+    this.indexKeyFields = config.getString(FlinkOptions.INDEX_KEY_FIELD);
+    this.taskID = getRuntimeContext().getIndexOfThisSubtask();
+    this.currentParallelism = getRuntimeContext().getNumberOfParallelSubtasks();
+    this.taskTotalParallelism = getRuntimeContext().getMaxNumberOfParallelSubtasks();
+    bootstrapIndex();
+  }
+
+  @Override
+  public void initializeState(FunctionInitializationContext context) throws Exception {
+    super.initializeState(context);
+    this.table = this.writeClient.getHoodieTable();
+  }
+
+  @Override
+  public void processElement(I i, ProcessFunction<I, Object>.Context context, Collector<Object> collector) throws Exception {
+    HoodieRecord<?> record = (HoodieRecord<?>) i;
+    final HoodieKey hoodieKey = record.getKey();
+    final HoodieRecordLocation location;
+
+    final int bucketNum = BucketIdentifier.getBucketId(hoodieKey, indexKeyFields, hiveBucketNum);
+    final String partitionBucketId = BucketIdentifier.partitionBucketIdStr(hoodieKey.getPartitionPath(), bucketNum);
+
+    if (bucketToFileIDMap.containsKey(partitionBucketId)) {

Review comment:
       you can change the job parallelism, but you can't change the bucket index number at this point.




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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] mincwang commented on pull request #4679: [RFC-35] Make Flink writer stream friendly

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


   Wow~ Great


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] vinothchandar commented on pull request #4679: [HUDI-3315] RFC-35 Make Flink writer stream friendly

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


   cc @yihua could you also please review this from the angle of making the write client abstractions more friendly


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] minihippo commented on a change in pull request #4679: [HUDI-3315] RFC-35 Part-1 Support bucket index in Flink writer

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



##########
File path: hudi-flink/src/main/java/org/apache/hudi/sink/BucketStreamWriteFunction.java
##########
@@ -0,0 +1,158 @@
+/*
+ * 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.configuration.Configuration;
+import org.apache.flink.runtime.state.FunctionInitializationContext;
+import org.apache.flink.streaming.api.functions.ProcessFunction;
+import org.apache.flink.util.Collector;
+import org.apache.hudi.common.model.FileSlice;
+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.table.timeline.HoodieInstant;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.configuration.FlinkOptions;
+import org.apache.hudi.index.bucket.BucketIdentifier;
+import org.apache.hudi.table.HoodieFlinkTable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+
+import static java.util.stream.Collectors.toList;
+
+public class BucketStreamWriteFunction<I> extends StreamWriteFunction<I> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BucketStreamWriteFunction.class);
+
+  private int maxParallelism;
+
+  private int parallelism;
+
+  private int hiveBucketNum;
+
+  protected transient HoodieFlinkTable table;
+
+  private String indexKeyFields;
+
+  private HashMap<String, String> bucketToFileIDMap;
+
+  /**
+   * Constructs a BucketStreamWriteFunction.
+   *
+   * @param config The config options
+   */
+  public BucketStreamWriteFunction(Configuration config) {
+    super(config);
+    this.bucketToFileIDMap = new HashMap<String, String>();
+  }
+
+  @Override
+  public void open(Configuration parameters) throws IOException {
+    super.open(parameters);
+    this.hiveBucketNum = config.getInteger(FlinkOptions.BUCKET_INDEX_NUM_BUCKETS);
+    this.indexKeyFields = config.getString(FlinkOptions.INDEX_KEY_FIELD);
+    this.taskID = getRuntimeContext().getIndexOfThisSubtask();
+    this.parallelism = getRuntimeContext().getNumberOfParallelSubtasks();
+    this.maxParallelism = getRuntimeContext().getMaxNumberOfParallelSubtasks();
+    bootstrapIndex();
+  }
+
+  @Override
+  public void initializeState(FunctionInitializationContext context) throws Exception {
+    super.initializeState(context);
+    this.table = this.writeClient.getHoodieTable();
+  }
+
+  @Override
+  public void processElement(I i, ProcessFunction<I, Object>.Context context, Collector<Object> collector) throws Exception {
+    HoodieRecord<?> record = (HoodieRecord<?>) i;
+    final HoodieKey hoodieKey = record.getKey();
+    final HoodieRecordLocation location;
+
+    final int bucketNum = BucketIdentifier.getBucketId(hoodieKey, indexKeyFields, hiveBucketNum);
+    final String partitionBucketId = BucketIdentifier.partitionBucketIdStr(hoodieKey.getPartitionPath(), bucketNum);
+
+    if (bucketToFileIDMap.containsKey(partitionBucketId)) {
+      location = new HoodieRecordLocation("U", bucketToFileIDMap.get(partitionBucketId));
+    } else {
+      String newFileId = BucketIdentifier.newBucketFileIdPrefix(bucketNum);
+      location = new HoodieRecordLocation("I", newFileId);
+      bucketToFileIDMap.put(partitionBucketId, newFileId);
+    }
+    record.unseal();
+    record.setCurrentLocation(location);
+    record.seal();
+    bufferRecord(record);
+  }
+
+  /**
+   * Get partition_bucket -> fileID mapping from the existing hudi table.
+   * This is a required operation for each restart to avoid having duplicate file ids for one bucket.
+   */
+  private void bootstrapIndex() throws IOException {
+    Option<HoodieInstant> latestCommitTime = table.getFileSystemView().getTimeline().filterCompletedInstants().lastInstant();
+    if (!latestCommitTime.isPresent()) {
+      return;
+    }
+    // bootstrap bucket info from existing file system
+    // bucketNum % totalParallelism == this taskID belongs to this task
+    HashSet<Integer> bucketToLoad = new HashSet<>();
+    for (int i = 0; i < hiveBucketNum; i++) {
+      int partitionOfBucket = BucketIdentifier.mod(i, parallelism);
+      if (partitionOfBucket == taskID) {
+        LOG.info(String.format("Bootstrapping index. Adding bucket %s , "
+            + "Current parallelism: %s , Max parallelism: %s , Current task id: %s",
+            i, parallelism, maxParallelism, taskID));
+        bucketToLoad.add(i);
+      }
+    }
+    bucketToLoad.stream().forEach(bucket -> LOG.info(String.format("bucketToLoad contains %s", bucket)));
+
+    LOG.info(String.format("Loading Hoodie Table %s, with path %s", table.getMetaClient().getTableConfig().getTableName(),
+        table.getMetaClient().getBasePath()));
+
+    // Iterate through all existing partitions to load existing fileID belongs to this task
+    List<String> partitions = table.getMetadata().getAllPartitionPaths();

Review comment:
       fix it with  https://github.com/apache/hudi/pull/5093




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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] allenxyang commented on a change in pull request #4679: [HUDI-3315] RFC-35 Part-1 Support bucket index in Flink writer

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



##########
File path: hudi-flink/src/main/java/org/apache/hudi/sink/utils/Pipelines.java
##########
@@ -189,21 +191,32 @@
   }
 
   public static DataStream<Object> hoodieStreamWrite(Configuration conf, int defaultParallelism, DataStream<HoodieRecord> dataStream) {
-    WriteOperatorFactory<HoodieRecord> operatorFactory = StreamWriteOperator.getFactory(conf);
-    return dataStream
+    if (OptionsResolver.isBucketIndexType(conf)) {

Review comment:
       After I used this patch, reported this error. I don't know why.
   
   Caused by: java.lang.RuntimeException: The timer service has not been initialized.
   	at org.apache.flink.streaming.api.operators.AbstractStreamOperator.getInternalTimerService(AbstractStreamOperator.java:616) 
   	at org.apache.flink.streaming.api.operators.KeyedProcessOperator.open(KeyedProcessOperator.java:62)
   	at org.apache.flink.streaming.runtime.tasks.OperatorChain.initializeStateAndOpenOperators(OperatorChain.java:428)
   	at org.apache.flink.streaming.runtime.tasks.StreamTask.lambda$beforeInvoke$2(StreamTask.java:555) 
   	at org.apache.flink.streaming.runtime.tasks.StreamTaskActionExecutor$1.runThrowing(StreamTaskActionExecutor.java:50) 
   	at org.apache.flink.streaming.runtime.tasks.StreamTask.beforeInvoke(StreamTask.java:545)
   	at org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:585) 
   	at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:765)
   	at org.apache.flink.runtime.taskmanager.Task.run(Task.java:580)




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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] hudi-bot removed a comment on pull request #4679: [HUDI-3315] RFC-35 Part-1 Support bucket index in Flink writer

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5471",
       "triggerID" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5491",
       "triggerID" : "1020748430",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5492",
       "triggerID" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5493",
       "triggerID" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5528",
       "triggerID" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6056",
       "triggerID" : "1041367799",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6056",
       "triggerID" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1041367799",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6189",
       "triggerID" : "1047380901",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6189",
       "triggerID" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1047380901",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6195",
       "triggerID" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6196",
       "triggerID" : "1047494529",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6195",
       "triggerID" : "1047494529",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6271",
       "triggerID" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6272",
       "triggerID" : "1049827248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6271",
       "triggerID" : "1049827248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6299",
       "triggerID" : "1050610982",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "8120f64681ac4e42a075d2cc04b036f737622ccd",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6388",
       "triggerID" : "8120f64681ac4e42a075d2cc04b036f737622ccd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8120f64681ac4e42a075d2cc04b036f737622ccd",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6390",
       "triggerID" : "1054218095",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "8120f64681ac4e42a075d2cc04b036f737622ccd",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6388",
       "triggerID" : "1054218095",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * 8120f64681ac4e42a075d2cc04b036f737622ccd Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6390) Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6388) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] hudi-bot removed a comment on pull request #4679: [HUDI-3315] RFC-35 Part-1 Support bucket index in Flink writer

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5471",
       "triggerID" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5491",
       "triggerID" : "1020748430",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5492",
       "triggerID" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5493",
       "triggerID" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5528",
       "triggerID" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6056",
       "triggerID" : "1041367799",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6056",
       "triggerID" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1041367799",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6189",
       "triggerID" : "1047380901",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6189",
       "triggerID" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1047380901",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6195",
       "triggerID" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6196",
       "triggerID" : "1047494529",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6195",
       "triggerID" : "1047494529",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6271",
       "triggerID" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6272",
       "triggerID" : "1049827248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6271",
       "triggerID" : "1049827248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6299",
       "triggerID" : "1050610982",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "8120f64681ac4e42a075d2cc04b036f737622ccd",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8120f64681ac4e42a075d2cc04b036f737622ccd",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6272) Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6271) Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6299) 
   * 8120f64681ac4e42a075d2cc04b036f737622ccd UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] danny0405 commented on a change in pull request #4679: [HUDI-3315] RFC-35 Make Flink writer stream friendly

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



##########
File path: hudi-flink/src/main/java/org/apache/hudi/table/HoodieTableSink.java
##########
@@ -84,12 +85,20 @@ public SinkRuntimeProvider getSinkRuntimeProvider(Context context) {
       // default parallelism
       int parallelism = dataStream.getExecutionConfig().getParallelism();
       DataStream<Object> pipeline;
-
-      // bootstrap
-      final DataStream<HoodieRecord> hoodieRecordDataStream =
-          Pipelines.bootstrap(conf, rowType, parallelism, dataStream, context.isBounded(), overwrite);
-      // write pipeline
-      pipeline = Pipelines.hoodieStreamWrite(conf, parallelism, hoodieRecordDataStream);
+      if (OptionsResolver.isBucketIndexTable(conf)) {
+        if (!OptionsResolver.isMorTable(conf)) {
+          throw new HoodieNotSupportedException("Bucket index only support MOR table type.");

Review comment:
       We can move the whole `if ... else ...` code block into the ` Pipelines.hoodieStreamWrite` function.




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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] hudi-bot commented on pull request #4679: [HUDI-3315] RFC-35 Part-1 Support bucket index in Flink writer

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5471",
       "triggerID" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5491",
       "triggerID" : "1020748430",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5492",
       "triggerID" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5493",
       "triggerID" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5528",
       "triggerID" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6056",
       "triggerID" : "1041367799",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6056",
       "triggerID" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1041367799",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6189",
       "triggerID" : "1047380901",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6189",
       "triggerID" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1047380901",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6195",
       "triggerID" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6196",
       "triggerID" : "1047494529",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6195",
       "triggerID" : "1047494529",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 2e89b137fc1ede6cc5bd95fdf2e414eea706b091 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6196) Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6195) 
   * 96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] danny0405 commented on a change in pull request #4679: [HUDI-3315] RFC-35 Part-1 Support bucket index in Flink writer

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



##########
File path: hudi-flink/src/main/java/org/apache/hudi/sink/BucketStreamWriteFunction.java
##########
@@ -0,0 +1,158 @@
+/*
+ * 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.configuration.Configuration;
+import org.apache.flink.runtime.state.FunctionInitializationContext;
+import org.apache.flink.streaming.api.functions.ProcessFunction;
+import org.apache.flink.util.Collector;
+import org.apache.hudi.common.model.FileSlice;
+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.table.timeline.HoodieInstant;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.configuration.FlinkOptions;
+import org.apache.hudi.index.bucket.BucketIdentifier;
+import org.apache.hudi.table.HoodieFlinkTable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+
+import static java.util.stream.Collectors.toList;
+
+public class BucketStreamWriteFunction<I> extends StreamWriteFunction<I> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BucketStreamWriteFunction.class);
+
+  private int taskTotalParallelism;
+
+  private int currentParallelism;
+
+  private int hiveBucketNum;
+
+  protected transient HoodieFlinkTable table;
+
+  private String indexKeyFields;
+
+  private HashMap<String, String> bucketToFileIDMap;
+
+  /**
+   * Constructs a BucketStreamWriteFunction.
+   *
+   * @param config The config options
+   */
+  public BucketStreamWriteFunction(Configuration config) {
+    super(config);
+    this.bucketToFileIDMap = new HashMap<String, String>();
+  }
+
+  @Override
+  public void open(Configuration parameters) throws IOException {
+    super.open(parameters);
+    this.hiveBucketNum = config.getInteger(FlinkOptions.BUCKET_INDEX_NUM_BUCKETS);
+    this.indexKeyFields = config.getString(FlinkOptions.INDEX_KEY_FIELD);
+    this.taskID = getRuntimeContext().getIndexOfThisSubtask();
+    this.currentParallelism = getRuntimeContext().getNumberOfParallelSubtasks();
+    this.taskTotalParallelism = getRuntimeContext().getMaxNumberOfParallelSubtasks();
+    bootstrapIndex();
+  }
+
+  @Override
+  public void initializeState(FunctionInitializationContext context) throws Exception {
+    super.initializeState(context);
+    this.table = this.writeClient.getHoodieTable();
+  }
+
+  @Override
+  public void processElement(I i, ProcessFunction<I, Object>.Context context, Collector<Object> collector) throws Exception {
+    HoodieRecord<?> record = (HoodieRecord<?>) i;
+    final HoodieKey hoodieKey = record.getKey();
+    final HoodieRecordLocation location;
+
+    final int bucketNum = BucketIdentifier.getBucketId(hoodieKey, indexKeyFields, hiveBucketNum);
+    final String partitionBucketId = BucketIdentifier.partitionBucketIdStr(hoodieKey.getPartitionPath(), bucketNum);
+
+    if (bucketToFileIDMap.containsKey(partitionBucketId)) {

Review comment:
       I guess @loukey-lj want to address that when the record switches to new partition, how we send a delete record to the old partition ?




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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] hudi-bot removed a comment on pull request #4679: [HUDI-3315] RFC-35 Make Flink writer stream friendly

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5471",
       "triggerID" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5491",
       "triggerID" : "1020748430",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5492",
       "triggerID" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5493",
       "triggerID" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5528",
       "triggerID" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "CANCELED",
       "url" : "TBD",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   *  Unknown: [CANCELED](TBD) 
   * f63a0753eb2f8f25cd1cd68bf5e6356a73438588 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] hudi-bot removed a comment on pull request #4679: [HUDI-3315] RFC-35 Make Flink writer stream friendly

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5471",
       "triggerID" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5491",
       "triggerID" : "1020748430",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5492",
       "triggerID" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5493",
       "triggerID" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5528",
       "triggerID" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * f63a0753eb2f8f25cd1cd68bf5e6356a73438588 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] hudi-bot removed a comment on pull request #4679: [HUDI-3315] RFC-35 Make Flink writer stream friendly

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






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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] hudi-bot commented on pull request #4679: [HUDI-3315] RFC-35 Make Flink writer stream friendly

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5471",
       "triggerID" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5491",
       "triggerID" : "1020748430",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5492",
       "triggerID" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5493",
       "triggerID" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5528",
       "triggerID" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "1041367799",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "",
       "status" : "CANCELED",
       "url" : "TBD",
       "triggerID" : "1041367799",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   *  Unknown: [CANCELED](TBD) 
   * ddbf502d11717d800971f736fa81054afa1873b7 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] hudi-bot removed a comment on pull request #4679: [HUDI-3315] RFC-35 Make Flink writer stream friendly

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5471",
       "triggerID" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5491",
       "triggerID" : "1020748430",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5492",
       "triggerID" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5493",
       "triggerID" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5528",
       "triggerID" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "",
       "status" : "CANCELED",
       "url" : "TBD",
       "triggerID" : "1041367799",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6056",
       "triggerID" : "1041367799",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6056",
       "triggerID" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   *  Unknown: [CANCELED](TBD) 
   * ddbf502d11717d800971f736fa81054afa1873b7 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6056) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] hudi-bot commented on pull request #4679: [HUDI-3315] RFC-35 Make Flink writer stream friendly

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5471",
       "triggerID" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5491",
       "triggerID" : "1020748430",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5492",
       "triggerID" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5493",
       "triggerID" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5528",
       "triggerID" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * f63a0753eb2f8f25cd1cd68bf5e6356a73438588 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] garyli1019 commented on pull request #4679: [HUDI-3315] RFC-35 Make Flink writer stream friendly

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


   @hudi-bot run azure


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] hudi-bot commented on pull request #4679: [HUDI-3315] RFC-35 Make Flink writer stream friendly

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5471",
       "triggerID" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5491",
       "triggerID" : "1020748430",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5492",
       "triggerID" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5493",
       "triggerID" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5528",
       "triggerID" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6056",
       "triggerID" : "1041367799",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6056",
       "triggerID" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1041367799",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6189",
       "triggerID" : "1047380901",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6189",
       "triggerID" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1047380901",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * 48b162c6c411b9cb5b605d006acc8d6e7d45cec8 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6189) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] hudi-bot removed a comment on pull request #4679: [HUDI-3315] RFC-35 Part-1 Support bucket index in Flink writer

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5471",
       "triggerID" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5491",
       "triggerID" : "1020748430",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5492",
       "triggerID" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5493",
       "triggerID" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5528",
       "triggerID" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6056",
       "triggerID" : "1041367799",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6056",
       "triggerID" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1041367799",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6189",
       "triggerID" : "1047380901",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6189",
       "triggerID" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1047380901",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6195",
       "triggerID" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6196",
       "triggerID" : "1047494529",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6195",
       "triggerID" : "1047494529",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * 2e89b137fc1ede6cc5bd95fdf2e414eea706b091 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6196) Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6195) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] hudi-bot removed a comment on pull request #4679: [HUDI-3315] RFC-35 Part-1 Support bucket index in Flink writer

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5471",
       "triggerID" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5491",
       "triggerID" : "1020748430",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5492",
       "triggerID" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5493",
       "triggerID" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5528",
       "triggerID" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6056",
       "triggerID" : "1041367799",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6056",
       "triggerID" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1041367799",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6189",
       "triggerID" : "1047380901",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6189",
       "triggerID" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1047380901",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6195",
       "triggerID" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6196",
       "triggerID" : "1047494529",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6195",
       "triggerID" : "1047494529",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 2e89b137fc1ede6cc5bd95fdf2e414eea706b091 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6196) Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6195) 
   * 96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] danny0405 commented on a change in pull request #4679: [HUDI-3315] RFC-35 Part-1 Support bucket index in Flink writer

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



##########
File path: hudi-flink/src/main/java/org/apache/hudi/sink/BucketStreamWriteFunction.java
##########
@@ -0,0 +1,158 @@
+/*
+ * 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.configuration.Configuration;
+import org.apache.flink.runtime.state.FunctionInitializationContext;
+import org.apache.flink.streaming.api.functions.ProcessFunction;
+import org.apache.flink.util.Collector;
+import org.apache.hudi.common.model.FileSlice;
+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.table.timeline.HoodieInstant;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.configuration.FlinkOptions;
+import org.apache.hudi.index.bucket.BucketIdentifier;
+import org.apache.hudi.table.HoodieFlinkTable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+
+import static java.util.stream.Collectors.toList;
+
+public class BucketStreamWriteFunction<I> extends StreamWriteFunction<I> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BucketStreamWriteFunction.class);
+
+  private int maxParallelism;
+
+  private int parallelism;
+
+  private int hiveBucketNum;
+
+  protected transient HoodieFlinkTable table;
+
+  private String indexKeyFields;
+
+  private HashMap<String, String> bucketToFileIDMap;
+
+  /**
+   * Constructs a BucketStreamWriteFunction.
+   *
+   * @param config The config options
+   */
+  public BucketStreamWriteFunction(Configuration config) {
+    super(config);
+    this.bucketToFileIDMap = new HashMap<String, String>();
+  }
+
+  @Override
+  public void open(Configuration parameters) throws IOException {
+    super.open(parameters);
+    this.hiveBucketNum = config.getInteger(FlinkOptions.BUCKET_INDEX_NUM_BUCKETS);
+    this.indexKeyFields = config.getString(FlinkOptions.INDEX_KEY_FIELD);
+    this.taskID = getRuntimeContext().getIndexOfThisSubtask();
+    this.parallelism = getRuntimeContext().getNumberOfParallelSubtasks();
+    this.maxParallelism = getRuntimeContext().getMaxNumberOfParallelSubtasks();
+    bootstrapIndex();
+  }
+
+  @Override
+  public void initializeState(FunctionInitializationContext context) throws Exception {
+    super.initializeState(context);
+    this.table = this.writeClient.getHoodieTable();
+  }
+
+  @Override
+  public void processElement(I i, ProcessFunction<I, Object>.Context context, Collector<Object> collector) throws Exception {
+    HoodieRecord<?> record = (HoodieRecord<?>) i;
+    final HoodieKey hoodieKey = record.getKey();
+    final HoodieRecordLocation location;
+
+    final int bucketNum = BucketIdentifier.getBucketId(hoodieKey, indexKeyFields, hiveBucketNum);
+    final String partitionBucketId = BucketIdentifier.partitionBucketIdStr(hoodieKey.getPartitionPath(), bucketNum);
+
+    if (bucketToFileIDMap.containsKey(partitionBucketId)) {
+      location = new HoodieRecordLocation("U", bucketToFileIDMap.get(partitionBucketId));
+    } else {
+      String newFileId = BucketIdentifier.newBucketFileIdPrefix(bucketNum);
+      location = new HoodieRecordLocation("I", newFileId);
+      bucketToFileIDMap.put(partitionBucketId, newFileId);
+    }
+    record.unseal();
+    record.setCurrentLocation(location);
+    record.seal();
+    bufferRecord(record);
+  }
+
+  /**
+   * Get partition_bucket -> fileID mapping from the existing hudi table.
+   * This is a required operation for each restart to avoid having duplicate file ids for one bucket.
+   */
+  private void bootstrapIndex() throws IOException {
+    Option<HoodieInstant> latestCommitTime = table.getFileSystemView().getTimeline().filterCompletedInstants().lastInstant();
+    if (!latestCommitTime.isPresent()) {
+      return;
+    }
+    // bootstrap bucket info from existing file system
+    // bucketNum % totalParallelism == this taskID belongs to this task
+    HashSet<Integer> bucketToLoad = new HashSet<>();
+    for (int i = 0; i < hiveBucketNum; i++) {
+      int partitionOfBucket = BucketIdentifier.mod(i, parallelism);
+      if (partitionOfBucket == taskID) {
+        LOG.info(String.format("Bootstrapping index. Adding bucket %s , "
+            + "Current parallelism: %s , Max parallelism: %s , Current task id: %s",
+            i, parallelism, maxParallelism, taskID));
+        bucketToLoad.add(i);
+      }
+    }
+    bucketToLoad.stream().forEach(bucket -> LOG.info(String.format("bucketToLoad contains %s", bucket)));
+
+    LOG.info(String.format("Loading Hoodie Table %s, with path %s", table.getMetaClient().getTableConfig().getTableName(),
+        table.getMetaClient().getBasePath()));
+
+    // Iterate through all existing partitions to load existing fileID belongs to this task
+    List<String> partitions = table.getMetadata().getAllPartitionPaths();

Review comment:
       Thanks, can we fire an issue to address this improvement ?




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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] mincwang commented on pull request #4679: [RFC-35] Make Flink writer stream friendly

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


   Wow~ Great


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] hudi-bot commented on pull request #4679: [HUDI-2450] RFC-35 Make Flink writer stream friendly

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5471",
       "triggerID" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5491",
       "triggerID" : "1020748430",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5492",
       "triggerID" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d4f6e12cca613145904db85abc3e1bd68a08e823 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5471) Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5491) 
   * 919d42914185e265b21d6124090bbda7a882f0f9 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5492) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] hudi-bot commented on pull request #4679: [HUDI-3315] RFC-35 Make Flink writer stream friendly

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5471",
       "triggerID" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5491",
       "triggerID" : "1020748430",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5492",
       "triggerID" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5493",
       "triggerID" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 7810bd0c529dc44094e13650696d585d21b23c82 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5493) 
   * 4d49dbe71e689f0d56d3447bc5c4367db35c33af UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] hudi-bot removed a comment on pull request #4679: [RFC-35] Make Flink writer stream friendly

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






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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] hudi-bot removed a comment on pull request #4679: [HUDI-2450] RFC-35 Make Flink writer stream friendly

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5471",
       "triggerID" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5491",
       "triggerID" : "1020748430",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * d4f6e12cca613145904db85abc3e1bd68a08e823 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5471) Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5491) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] hudi-bot commented on pull request #4679: [HUDI-3315] RFC-35 Make Flink writer stream friendly

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5471",
       "triggerID" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5491",
       "triggerID" : "1020748430",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5492",
       "triggerID" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5493",
       "triggerID" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 7810bd0c529dc44094e13650696d585d21b23c82 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5493) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] hudi-bot removed a comment on pull request #4679: [HUDI-3315] RFC-35 Part-1 Support bucket index in Flink writer

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5471",
       "triggerID" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5491",
       "triggerID" : "1020748430",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5492",
       "triggerID" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5493",
       "triggerID" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5528",
       "triggerID" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6056",
       "triggerID" : "1041367799",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6056",
       "triggerID" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1041367799",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6189",
       "triggerID" : "1047380901",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6189",
       "triggerID" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1047380901",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6195",
       "triggerID" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6196",
       "triggerID" : "1047494529",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6195",
       "triggerID" : "1047494529",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6271",
       "triggerID" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6272",
       "triggerID" : "1049827248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6271",
       "triggerID" : "1049827248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6299",
       "triggerID" : "1050610982",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * 96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6272) Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6271) Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6299) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] garyli1019 commented on a change in pull request #4679: [HUDI-3315] RFC-35 Part-1 Support bucket index in Flink writer

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



##########
File path: hudi-flink/src/main/java/org/apache/hudi/configuration/FlinkOptions.java
##########
@@ -310,6 +318,20 @@ private FlinkOptions() {
           + "Actual value will be obtained by invoking .toString() on the field value. Nested fields can be specified using "
           + "the dot notation eg: `a.b.c`");
 
+  public static final ConfigOption<String> INDEX_KEY_FIELD = ConfigOptions
+      .key(HoodieIndexConfig.BUCKET_INDEX_HASH_FIELD.key())
+      .stringType()
+      .defaultValue("uuid")

Review comment:
       hmm not quite sure we should do that here. The user need some basic knowledge about this index type, so they know how to operate the table. e.g. how to handle the bucket number and how does this hash key field work.

##########
File path: hudi-flink/src/main/java/org/apache/hudi/sink/BucketStreamWriteFunction.java
##########
@@ -0,0 +1,158 @@
+/*
+ * 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.configuration.Configuration;
+import org.apache.flink.runtime.state.FunctionInitializationContext;
+import org.apache.flink.streaming.api.functions.ProcessFunction;
+import org.apache.flink.util.Collector;
+import org.apache.hudi.common.model.FileSlice;
+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.table.timeline.HoodieInstant;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.configuration.FlinkOptions;
+import org.apache.hudi.index.bucket.BucketIdentifier;
+import org.apache.hudi.table.HoodieFlinkTable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+
+import static java.util.stream.Collectors.toList;
+
+public class BucketStreamWriteFunction<I> extends StreamWriteFunction<I> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BucketStreamWriteFunction.class);
+
+  private int maxParallelism;
+
+  private int parallelism;
+
+  private int hiveBucketNum;

Review comment:
       fixed




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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] danny0405 merged pull request #4679: [HUDI-3315] RFC-35 Part-1 Support bucket index in Flink writer

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


   


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] hudi-bot commented on pull request #4679: [HUDI-3315] RFC-35 Make Flink writer stream friendly

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






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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] garyli1019 commented on pull request #4679: [HUDI-3315] RFC-35 Make Flink writer stream friendly

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


   @hudi-bot run azure


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] hudi-bot removed a comment on pull request #4679: [HUDI-3315] RFC-35 Make Flink writer stream friendly

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5471",
       "triggerID" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5491",
       "triggerID" : "1020748430",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5492",
       "triggerID" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5493",
       "triggerID" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5528",
       "triggerID" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6056",
       "triggerID" : "1041367799",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6056",
       "triggerID" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1041367799",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "1047380901",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "",
       "status" : "CANCELED",
       "url" : "TBD",
       "triggerID" : "1047380901",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   *  Unknown: [CANCELED](TBD) 
   * 48b162c6c411b9cb5b605d006acc8d6e7d45cec8 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] hudi-bot commented on pull request #4679: [HUDI-3315] RFC-35 Part-1 Support bucket index in Flink writer

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5471",
       "triggerID" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5491",
       "triggerID" : "1020748430",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5492",
       "triggerID" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5493",
       "triggerID" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5528",
       "triggerID" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6056",
       "triggerID" : "1041367799",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6056",
       "triggerID" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1041367799",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6189",
       "triggerID" : "1047380901",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6189",
       "triggerID" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1047380901",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6195",
       "triggerID" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6196",
       "triggerID" : "1047494529",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6195",
       "triggerID" : "1047494529",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6271",
       "triggerID" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6272",
       "triggerID" : "1049827248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6271",
       "triggerID" : "1049827248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6299",
       "triggerID" : "1050610982",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * 96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6272) Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6271) Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6299) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] minihippo commented on a change in pull request #4679: [HUDI-3315] RFC-35 Part-1 Support bucket index in Flink writer

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



##########
File path: hudi-flink/src/main/java/org/apache/hudi/sink/BucketStreamWriteFunction.java
##########
@@ -0,0 +1,158 @@
+/*
+ * 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.configuration.Configuration;
+import org.apache.flink.runtime.state.FunctionInitializationContext;
+import org.apache.flink.streaming.api.functions.ProcessFunction;
+import org.apache.flink.util.Collector;
+import org.apache.hudi.common.model.FileSlice;
+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.table.timeline.HoodieInstant;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.configuration.FlinkOptions;
+import org.apache.hudi.index.bucket.BucketIdentifier;
+import org.apache.hudi.table.HoodieFlinkTable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+
+import static java.util.stream.Collectors.toList;
+
+public class BucketStreamWriteFunction<I> extends StreamWriteFunction<I> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BucketStreamWriteFunction.class);
+
+  private int maxParallelism;
+
+  private int parallelism;
+
+  private int hiveBucketNum;
+
+  protected transient HoodieFlinkTable table;
+
+  private String indexKeyFields;
+
+  private HashMap<String, String> bucketToFileIDMap;
+
+  /**
+   * Constructs a BucketStreamWriteFunction.
+   *
+   * @param config The config options
+   */
+  public BucketStreamWriteFunction(Configuration config) {
+    super(config);
+    this.bucketToFileIDMap = new HashMap<String, String>();
+  }
+
+  @Override
+  public void open(Configuration parameters) throws IOException {
+    super.open(parameters);
+    this.hiveBucketNum = config.getInteger(FlinkOptions.BUCKET_INDEX_NUM_BUCKETS);
+    this.indexKeyFields = config.getString(FlinkOptions.INDEX_KEY_FIELD);
+    this.taskID = getRuntimeContext().getIndexOfThisSubtask();
+    this.parallelism = getRuntimeContext().getNumberOfParallelSubtasks();
+    this.maxParallelism = getRuntimeContext().getMaxNumberOfParallelSubtasks();
+    bootstrapIndex();
+  }
+
+  @Override
+  public void initializeState(FunctionInitializationContext context) throws Exception {
+    super.initializeState(context);
+    this.table = this.writeClient.getHoodieTable();
+  }
+
+  @Override
+  public void processElement(I i, ProcessFunction<I, Object>.Context context, Collector<Object> collector) throws Exception {
+    HoodieRecord<?> record = (HoodieRecord<?>) i;
+    final HoodieKey hoodieKey = record.getKey();
+    final HoodieRecordLocation location;
+
+    final int bucketNum = BucketIdentifier.getBucketId(hoodieKey, indexKeyFields, hiveBucketNum);
+    final String partitionBucketId = BucketIdentifier.partitionBucketIdStr(hoodieKey.getPartitionPath(), bucketNum);
+
+    if (bucketToFileIDMap.containsKey(partitionBucketId)) {
+      location = new HoodieRecordLocation("U", bucketToFileIDMap.get(partitionBucketId));
+    } else {
+      String newFileId = BucketIdentifier.newBucketFileIdPrefix(bucketNum);
+      location = new HoodieRecordLocation("I", newFileId);
+      bucketToFileIDMap.put(partitionBucketId, newFileId);
+    }
+    record.unseal();
+    record.setCurrentLocation(location);
+    record.seal();
+    bufferRecord(record);
+  }
+
+  /**
+   * Get partition_bucket -> fileID mapping from the existing hudi table.
+   * This is a required operation for each restart to avoid having duplicate file ids for one bucket.
+   */
+  private void bootstrapIndex() throws IOException {
+    Option<HoodieInstant> latestCommitTime = table.getFileSystemView().getTimeline().filterCompletedInstants().lastInstant();
+    if (!latestCommitTime.isPresent()) {
+      return;
+    }
+    // bootstrap bucket info from existing file system
+    // bucketNum % totalParallelism == this taskID belongs to this task
+    HashSet<Integer> bucketToLoad = new HashSet<>();
+    for (int i = 0; i < hiveBucketNum; i++) {
+      int partitionOfBucket = BucketIdentifier.mod(i, parallelism);

Review comment:
       just change to i%parallelism. Cause i can not be negative.

##########
File path: hudi-flink/src/main/java/org/apache/hudi/configuration/FlinkOptions.java
##########
@@ -310,6 +318,20 @@ private FlinkOptions() {
           + "Actual value will be obtained by invoking .toString() on the field value. Nested fields can be specified using "
           + "the dot notation eg: `a.b.c`");
 
+  public static final ConfigOption<String> INDEX_KEY_FIELD = ConfigOptions
+      .key(HoodieIndexConfig.BUCKET_INDEX_HASH_FIELD.key())
+      .stringType()
+      .defaultValue("uuid")

Review comment:
       If the index key field not set, use the record key field as default to make the configuration more user-friendly?

##########
File path: hudi-flink/src/main/java/org/apache/hudi/sink/BucketStreamWriteFunction.java
##########
@@ -0,0 +1,158 @@
+/*
+ * 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.configuration.Configuration;
+import org.apache.flink.runtime.state.FunctionInitializationContext;
+import org.apache.flink.streaming.api.functions.ProcessFunction;
+import org.apache.flink.util.Collector;
+import org.apache.hudi.common.model.FileSlice;
+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.table.timeline.HoodieInstant;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.configuration.FlinkOptions;
+import org.apache.hudi.index.bucket.BucketIdentifier;
+import org.apache.hudi.table.HoodieFlinkTable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+
+import static java.util.stream.Collectors.toList;
+
+public class BucketStreamWriteFunction<I> extends StreamWriteFunction<I> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BucketStreamWriteFunction.class);
+
+  private int maxParallelism;
+
+  private int parallelism;
+
+  private int hiveBucketNum;

Review comment:
       just bucketNum?

##########
File path: hudi-flink/src/main/java/org/apache/hudi/sink/BucketStreamWriteFunction.java
##########
@@ -0,0 +1,158 @@
+/*
+ * 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.configuration.Configuration;
+import org.apache.flink.runtime.state.FunctionInitializationContext;
+import org.apache.flink.streaming.api.functions.ProcessFunction;
+import org.apache.flink.util.Collector;
+import org.apache.hudi.common.model.FileSlice;
+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.table.timeline.HoodieInstant;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.configuration.FlinkOptions;
+import org.apache.hudi.index.bucket.BucketIdentifier;
+import org.apache.hudi.table.HoodieFlinkTable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+
+import static java.util.stream.Collectors.toList;
+
+public class BucketStreamWriteFunction<I> extends StreamWriteFunction<I> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BucketStreamWriteFunction.class);
+
+  private int maxParallelism;
+
+  private int parallelism;
+
+  private int hiveBucketNum;
+
+  protected transient HoodieFlinkTable table;
+
+  private String indexKeyFields;
+
+  private HashMap<String, String> bucketToFileIDMap;
+
+  /**
+   * Constructs a BucketStreamWriteFunction.
+   *
+   * @param config The config options
+   */
+  public BucketStreamWriteFunction(Configuration config) {
+    super(config);
+    this.bucketToFileIDMap = new HashMap<String, String>();
+  }
+
+  @Override
+  public void open(Configuration parameters) throws IOException {
+    super.open(parameters);
+    this.hiveBucketNum = config.getInteger(FlinkOptions.BUCKET_INDEX_NUM_BUCKETS);
+    this.indexKeyFields = config.getString(FlinkOptions.INDEX_KEY_FIELD);
+    this.taskID = getRuntimeContext().getIndexOfThisSubtask();
+    this.parallelism = getRuntimeContext().getNumberOfParallelSubtasks();
+    this.maxParallelism = getRuntimeContext().getMaxNumberOfParallelSubtasks();
+    bootstrapIndex();
+  }
+
+  @Override
+  public void initializeState(FunctionInitializationContext context) throws Exception {
+    super.initializeState(context);
+    this.table = this.writeClient.getHoodieTable();
+  }
+
+  @Override
+  public void processElement(I i, ProcessFunction<I, Object>.Context context, Collector<Object> collector) throws Exception {
+    HoodieRecord<?> record = (HoodieRecord<?>) i;
+    final HoodieKey hoodieKey = record.getKey();
+    final HoodieRecordLocation location;
+
+    final int bucketNum = BucketIdentifier.getBucketId(hoodieKey, indexKeyFields, hiveBucketNum);
+    final String partitionBucketId = BucketIdentifier.partitionBucketIdStr(hoodieKey.getPartitionPath(), bucketNum);
+
+    if (bucketToFileIDMap.containsKey(partitionBucketId)) {
+      location = new HoodieRecordLocation("U", bucketToFileIDMap.get(partitionBucketId));
+    } else {
+      String newFileId = BucketIdentifier.newBucketFileIdPrefix(bucketNum);
+      location = new HoodieRecordLocation("I", newFileId);
+      bucketToFileIDMap.put(partitionBucketId, newFileId);
+    }
+    record.unseal();
+    record.setCurrentLocation(location);
+    record.seal();
+    bufferRecord(record);
+  }
+
+  /**
+   * Get partition_bucket -> fileID mapping from the existing hudi table.
+   * This is a required operation for each restart to avoid having duplicate file ids for one bucket.
+   */
+  private void bootstrapIndex() throws IOException {
+    Option<HoodieInstant> latestCommitTime = table.getFileSystemView().getTimeline().filterCompletedInstants().lastInstant();
+    if (!latestCommitTime.isPresent()) {
+      return;
+    }
+    // bootstrap bucket info from existing file system
+    // bucketNum % totalParallelism == this taskID belongs to this task
+    HashSet<Integer> bucketToLoad = new HashSet<>();
+    for (int i = 0; i < hiveBucketNum; i++) {
+      int partitionOfBucket = BucketIdentifier.mod(i, parallelism);
+      if (partitionOfBucket == taskID) {
+        LOG.info(String.format("Bootstrapping index. Adding bucket %s , "
+            + "Current parallelism: %s , Max parallelism: %s , Current task id: %s",
+            i, parallelism, maxParallelism, taskID));
+        bucketToLoad.add(i);
+      }
+    }
+    bucketToLoad.stream().forEach(bucket -> LOG.info(String.format("bucketToLoad contains %s", bucket)));
+
+    LOG.info(String.format("Loading Hoodie Table %s, with path %s", table.getMetaClient().getTableConfig().getTableName(),
+        table.getMetaClient().getBasePath()));
+
+    // Iterate through all existing partitions to load existing fileID belongs to this task
+    List<String> partitions = table.getMetadata().getAllPartitionPaths();

Review comment:
       It may have poor performance for application starting when the partition num is huge. Load it at runtime may be better, especially in the case most of the partitions in the table are frozen.




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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] garyli1019 commented on pull request #4679: [HUDI-3315] RFC-35 Part-1 Support bucket index in Flink writer

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


   @danny0405 If people change the write parallelism, it will still work because we load the parallelism-bucketID mapping at the runtime, but the parallelism should be less than the bucket number to avoid empty task. 
   At this point, the bucket number could not be changed. @minihippo is working on to support changing this number.


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] hudi-bot removed a comment on pull request #4679: [HUDI-3315] RFC-35 Part-1 Support bucket index in Flink writer

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5471",
       "triggerID" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5491",
       "triggerID" : "1020748430",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5492",
       "triggerID" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5493",
       "triggerID" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5528",
       "triggerID" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6056",
       "triggerID" : "1041367799",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6056",
       "triggerID" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1041367799",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6189",
       "triggerID" : "1047380901",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6189",
       "triggerID" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1047380901",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6195",
       "triggerID" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6196",
       "triggerID" : "1047494529",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6195",
       "triggerID" : "1047494529",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6271",
       "triggerID" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6272",
       "triggerID" : "1049827248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6271",
       "triggerID" : "1049827248",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * 96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6272) Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6271) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] hudi-bot commented on pull request #4679: [HUDI-3315] RFC-35 Make Flink writer stream friendly

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5471",
       "triggerID" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5491",
       "triggerID" : "1020748430",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5492",
       "triggerID" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5493",
       "triggerID" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5528",
       "triggerID" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "CANCELED",
       "url" : "TBD",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   *  Unknown: [CANCELED](TBD) 
   * f63a0753eb2f8f25cd1cd68bf5e6356a73438588 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] hudi-bot removed a comment on pull request #4679: [HUDI-3315] RFC-35 Make Flink writer stream friendly

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5471",
       "triggerID" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5491",
       "triggerID" : "1020748430",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5492",
       "triggerID" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 919d42914185e265b21d6124090bbda7a882f0f9 Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5492) 
   * 7810bd0c529dc44094e13650696d585d21b23c82 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] danny0405 commented on a change in pull request #4679: [HUDI-3315] RFC-35 Make Flink writer stream friendly

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



##########
File path: hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/HoodieFlinkWriteClient.java
##########
@@ -96,6 +101,34 @@
   public HoodieFlinkWriteClient(HoodieEngineContext context, HoodieWriteConfig writeConfig) {
     super(context, writeConfig);
     this.bucketToHandles = new HashMap<>();
+    this.bucketToAppendHandles = new HashMap<>();
+  }
+
+  public void init(HoodieFlinkTable<T> table) {
+    table.validateUpsertSchema();
+    setOperationType(WriteOperationType.UPSERT);
+  }
+
+  public Boolean buffer(HoodieRecord<T> record, String instantTime, HoodieFlinkTable<T> table) {
+    final HoodieRecordLocation loc = record.getCurrentLocation();
+    final String fileID = loc.getFileId();
+    final String partitionPath = record.getPartitionPath();

Review comment:
       Can we find a way to move the buffering logic into the write function ?




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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] yihua commented on pull request #4679: [HUDI-3315] RFC-35 Make Flink writer stream friendly

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


   > cc @yihua could you also please review this from the angle of making the write client abstractions more friendly
   
   I'll review this.


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] garyli1019 commented on pull request #4679: [HUDI-3315] RFC-35 Part-1 Support bucket index in Flink writer

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


   @hudi-bot run azure


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] garyli1019 commented on a change in pull request #4679: [HUDI-3315] RFC-35 Part-1 Support bucket index in Flink writer

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



##########
File path: hudi-flink/src/main/java/org/apache/hudi/sink/BucketStreamWriteFunction.java
##########
@@ -0,0 +1,158 @@
+/*
+ * 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.configuration.Configuration;
+import org.apache.flink.runtime.state.FunctionInitializationContext;
+import org.apache.flink.streaming.api.functions.ProcessFunction;
+import org.apache.flink.util.Collector;
+import org.apache.hudi.common.model.FileSlice;
+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.table.timeline.HoodieInstant;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.configuration.FlinkOptions;
+import org.apache.hudi.index.bucket.BucketIdentifier;
+import org.apache.hudi.table.HoodieFlinkTable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+
+import static java.util.stream.Collectors.toList;
+
+public class BucketStreamWriteFunction<I> extends StreamWriteFunction<I> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BucketStreamWriteFunction.class);
+
+  private int taskTotalParallelism;
+
+  private int currentParallelism;
+
+  private int hiveBucketNum;
+
+  protected transient HoodieFlinkTable table;
+
+  private String indexKeyFields;
+
+  private HashMap<String, String> bucketToFileIDMap;
+
+  /**
+   * Constructs a BucketStreamWriteFunction.
+   *
+   * @param config The config options
+   */
+  public BucketStreamWriteFunction(Configuration config) {
+    super(config);
+    this.bucketToFileIDMap = new HashMap<String, String>();
+  }
+
+  @Override
+  public void open(Configuration parameters) throws IOException {
+    super.open(parameters);
+    this.hiveBucketNum = config.getInteger(FlinkOptions.BUCKET_INDEX_NUM_BUCKETS);
+    this.indexKeyFields = config.getString(FlinkOptions.INDEX_KEY_FIELD);
+    this.taskID = getRuntimeContext().getIndexOfThisSubtask();
+    this.currentParallelism = getRuntimeContext().getNumberOfParallelSubtasks();
+    this.taskTotalParallelism = getRuntimeContext().getMaxNumberOfParallelSubtasks();
+    bootstrapIndex();

Review comment:
       fixed

##########
File path: hudi-flink/src/main/java/org/apache/hudi/sink/common/AbstractStreamWriteFunction.java
##########
@@ -107,7 +107,7 @@
   /**
    * List state of the write metadata events.
    */
-  private transient ListState<WriteMetadataEvent> writeMetadataState;
+  protected transient ListState<WriteMetadataEvent> writeMetadataState;
 

Review comment:
       fixed




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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] hudi-bot commented on pull request #4679: [HUDI-3315] RFC-35 Part-1 Support bucket index in Flink writer

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5471",
       "triggerID" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5491",
       "triggerID" : "1020748430",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5492",
       "triggerID" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5493",
       "triggerID" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5528",
       "triggerID" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6056",
       "triggerID" : "1041367799",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6056",
       "triggerID" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1041367799",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6189",
       "triggerID" : "1047380901",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6189",
       "triggerID" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1047380901",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6195",
       "triggerID" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6196",
       "triggerID" : "1047494529",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6195",
       "triggerID" : "1047494529",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6271",
       "triggerID" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6272",
       "triggerID" : "1049827248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6271",
       "triggerID" : "1049827248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6299",
       "triggerID" : "1050610982",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "8120f64681ac4e42a075d2cc04b036f737622ccd",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6388",
       "triggerID" : "8120f64681ac4e42a075d2cc04b036f737622ccd",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6272) Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6271) Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6299) 
   * 8120f64681ac4e42a075d2cc04b036f737622ccd Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6388) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] hudi-bot commented on pull request #4679: [HUDI-3315] RFC-35 Part-1 Support bucket index in Flink writer

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5471",
       "triggerID" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5491",
       "triggerID" : "1020748430",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5492",
       "triggerID" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5493",
       "triggerID" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5528",
       "triggerID" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6056",
       "triggerID" : "1041367799",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6056",
       "triggerID" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1041367799",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6189",
       "triggerID" : "1047380901",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6189",
       "triggerID" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1047380901",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6195",
       "triggerID" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6196",
       "triggerID" : "1047494529",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6195",
       "triggerID" : "1047494529",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6271",
       "triggerID" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6272",
       "triggerID" : "1049827248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6271",
       "triggerID" : "1049827248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6299",
       "triggerID" : "1050610982",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "8120f64681ac4e42a075d2cc04b036f737622ccd",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8120f64681ac4e42a075d2cc04b036f737622ccd",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6272) Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6271) Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6299) 
   * 8120f64681ac4e42a075d2cc04b036f737622ccd UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] garyli1019 commented on a change in pull request #4679: [HUDI-3315] RFC-35 Part-1 Support bucket index in Flink writer

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



##########
File path: hudi-flink/src/main/java/org/apache/hudi/configuration/FlinkOptions.java
##########
@@ -310,6 +318,20 @@ private FlinkOptions() {
           + "Actual value will be obtained by invoking .toString() on the field value. Nested fields can be specified using "
           + "the dot notation eg: `a.b.c`");
 
+  public static final ConfigOption<String> INDEX_KEY_FIELD = ConfigOptions
+      .key(HoodieIndexConfig.BUCKET_INDEX_HASH_FIELD.key())
+      .stringType()

Review comment:
       this could be a subset of primary keys. e.g. primary key could be "id1,id2", this index key could be either "id1" "id2" "id1,id2".




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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] garyli1019 commented on a change in pull request #4679: [HUDI-3315] RFC-35 Part-1 Support bucket index in Flink writer

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



##########
File path: hudi-flink/src/main/java/org/apache/hudi/configuration/OptionsResolver.java
##########
@@ -101,6 +102,10 @@ public static boolean isPartitionedTable(Configuration conf) {
     return FilePathUtils.extractPartitionKeys(conf).length > 0;
   }
 
+  public static boolean isBucketIndexTable(Configuration conf) {
+    return conf.getString(FlinkOptions.INDEX_TYPE).equals(HoodieIndex.IndexType.BUCKET.name());

Review comment:
       fixed




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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] hudi-bot commented on pull request #4679: [HUDI-3315] RFC-35 Make Flink writer stream friendly

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5471",
       "triggerID" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5491",
       "triggerID" : "1020748430",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5492",
       "triggerID" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5493",
       "triggerID" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5528",
       "triggerID" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "",
       "status" : "CANCELED",
       "url" : "TBD",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   *  Unknown: [CANCELED](TBD) 
   * f63a0753eb2f8f25cd1cd68bf5e6356a73438588 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] garyli1019 commented on pull request #4679: [HUDI-3315] RFC-35 Make Flink writer stream friendly

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


   @hudi-bot run azure


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] hudi-bot commented on pull request #4679: [HUDI-3315] RFC-35 Part-1 Support bucket index in Flink writer

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5471",
       "triggerID" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5491",
       "triggerID" : "1020748430",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5492",
       "triggerID" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5493",
       "triggerID" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5528",
       "triggerID" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6056",
       "triggerID" : "1041367799",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6056",
       "triggerID" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1041367799",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6189",
       "triggerID" : "1047380901",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6189",
       "triggerID" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1047380901",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6195",
       "triggerID" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6196",
       "triggerID" : "1047494529",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6195",
       "triggerID" : "1047494529",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6271",
       "triggerID" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6272",
       "triggerID" : "1049827248",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * 2e89b137fc1ede6cc5bd95fdf2e414eea706b091 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6196) Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6195) 
   * 96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6271) Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6272) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] garyli1019 commented on a change in pull request #4679: [HUDI-3315] RFC-35 Part-1 Support bucket index in Flink writer

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



##########
File path: hudi-flink/src/main/java/org/apache/hudi/configuration/FlinkOptions.java
##########
@@ -106,6 +108,12 @@ private FlinkOptions() {
   // ------------------------------------------------------------------------
   //  Index Options
   // ------------------------------------------------------------------------
+  public static final ConfigOption<String> INDEX_TYPE = ConfigOptions
+      .key("write.index.type")
+      .stringType()

Review comment:
       fixed




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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] hudi-bot removed a comment on pull request #4679: [HUDI-3315] RFC-35 Make Flink writer stream friendly

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5471",
       "triggerID" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5491",
       "triggerID" : "1020748430",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5492",
       "triggerID" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5493",
       "triggerID" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5528",
       "triggerID" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6056",
       "triggerID" : "1041367799",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6056",
       "triggerID" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1041367799",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6189",
       "triggerID" : "1047380901",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6189",
       "triggerID" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1047380901",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 48b162c6c411b9cb5b605d006acc8d6e7d45cec8 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6189) 
   * 2e89b137fc1ede6cc5bd95fdf2e414eea706b091 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] hudi-bot commented on pull request #4679: [HUDI-3315] RFC-35 Make Flink writer stream friendly

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5471",
       "triggerID" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5491",
       "triggerID" : "1020748430",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5492",
       "triggerID" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5493",
       "triggerID" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5528",
       "triggerID" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6056",
       "triggerID" : "1041367799",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6056",
       "triggerID" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1041367799",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6189",
       "triggerID" : "1047380901",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6189",
       "triggerID" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1047380901",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6195",
       "triggerID" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6196",
       "triggerID" : "1047494529",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * 48b162c6c411b9cb5b605d006acc8d6e7d45cec8 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6189) 
   * 2e89b137fc1ede6cc5bd95fdf2e414eea706b091 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6195) Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6196) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] garyli1019 commented on a change in pull request #4679: [HUDI-3315] RFC-35 Part-1 Support bucket index in Flink writer

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



##########
File path: hudi-flink/src/main/java/org/apache/hudi/sink/BucketStreamWriteFunction.java
##########
@@ -0,0 +1,158 @@
+/*
+ * 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.configuration.Configuration;
+import org.apache.flink.runtime.state.FunctionInitializationContext;
+import org.apache.flink.streaming.api.functions.ProcessFunction;
+import org.apache.flink.util.Collector;
+import org.apache.hudi.common.model.FileSlice;
+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.table.timeline.HoodieInstant;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.configuration.FlinkOptions;
+import org.apache.hudi.index.bucket.BucketIdentifier;
+import org.apache.hudi.table.HoodieFlinkTable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+
+import static java.util.stream.Collectors.toList;
+
+public class BucketStreamWriteFunction<I> extends StreamWriteFunction<I> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BucketStreamWriteFunction.class);
+
+  private int taskTotalParallelism;
+
+  private int currentParallelism;
+
+  private int hiveBucketNum;
+
+  protected transient HoodieFlinkTable table;
+
+  private String indexKeyFields;
+
+  private HashMap<String, String> bucketToFileIDMap;
+
+  /**
+   * Constructs a BucketStreamWriteFunction.
+   *
+   * @param config The config options
+   */
+  public BucketStreamWriteFunction(Configuration config) {
+    super(config);
+    this.bucketToFileIDMap = new HashMap<String, String>();
+  }
+
+  @Override
+  public void open(Configuration parameters) throws IOException {
+    super.open(parameters);
+    this.hiveBucketNum = config.getInteger(FlinkOptions.BUCKET_INDEX_NUM_BUCKETS);
+    this.indexKeyFields = config.getString(FlinkOptions.INDEX_KEY_FIELD);
+    this.taskID = getRuntimeContext().getIndexOfThisSubtask();
+    this.currentParallelism = getRuntimeContext().getNumberOfParallelSubtasks();
+    this.taskTotalParallelism = getRuntimeContext().getMaxNumberOfParallelSubtasks();
+    bootstrapIndex();
+  }
+
+  @Override
+  public void initializeState(FunctionInitializationContext context) throws Exception {
+    super.initializeState(context);
+    this.table = this.writeClient.getHoodieTable();
+  }
+
+  @Override
+  public void processElement(I i, ProcessFunction<I, Object>.Context context, Collector<Object> collector) throws Exception {
+    HoodieRecord<?> record = (HoodieRecord<?>) i;
+    final HoodieKey hoodieKey = record.getKey();
+    final HoodieRecordLocation location;
+
+    final int bucketNum = BucketIdentifier.getBucketId(hoodieKey, indexKeyFields, hiveBucketNum);
+    final String partitionBucketId = BucketIdentifier.partitionBucketIdStr(hoodieKey.getPartitionPath(), bucketNum);
+
+    if (bucketToFileIDMap.containsKey(partitionBucketId)) {
+      location = new HoodieRecordLocation("U", bucketToFileIDMap.get(partitionBucketId));
+    } else {
+      String newFileId = BucketIdentifier.newBucketFileIdPrefix(bucketNum);
+      location = new HoodieRecordLocation("I", newFileId);
+      bucketToFileIDMap.put(partitionBucketId, newFileId);
+    }
+    record.unseal();
+    record.setCurrentLocation(location);
+    record.seal();
+    bufferRecord(record);
+  }
+
+  /**
+   * Get partition_bucket -> fileID mapping from the existing hudi table.
+   * This is a required operation for each restart to avoid having duplicate file ids for one bucket.
+   */
+  private void bootstrapIndex() throws IOException {
+    Option<HoodieInstant> latestCommitTime = table.getFileSystemView().getTimeline().filterCompletedInstants().lastInstant();
+    if (!latestCommitTime.isPresent()) {
+      return;
+    }
+    // bootstrap bucket info from existing file system
+    // bucketNum % totalParallelism == this taskID belongs to this task
+    HashSet<Integer> bucketToLoad = new HashSet<>();
+    for (int i = 0; i < hiveBucketNum; i++) {
+      int partitionOfBucket = BucketIdentifier.mod(i, currentParallelism);
+      if (partitionOfBucket == taskID) {
+        LOG.info(String.format("Bootstrapping index. Adding bucket %s , "
+            + "Current parallelism: %s , Max parallelism: %s , Current task id: %s",
+            i, currentParallelism, taskTotalParallelism, taskID));
+        bucketToLoad.add(i);
+      }
+    }
+    bucketToLoad.stream().forEach(bucket -> LOG.info(String.format("bucketToLoad contains %s", bucket)));
+
+    LOG.info(String.format("Loading Hoodie Table %s, with path %s", table.getMetaClient().getTableConfig().getTableName(),
+        table.getMetaClient().getBasePath()));
+
+    // Iterate through all existing partitions to load existing fileID belongs to this task
+    List<String> partitions = table.getMetadata().getAllPartitionPaths();
+    for (String partitionPath : partitions) {
+      List<FileSlice> latestFileSlices = table.getSliceView()
+          .getLatestFileSlices(partitionPath)
+          .collect(toList());
+      for (FileSlice fileslice : latestFileSlices) {
+        String fileID = fileslice.getFileId();
+        int bucketNumber = BucketIdentifier.bucketIdFromFileId(fileID);
+        if (bucketToLoad.contains(bucketNumber)) {
+          String partitionBucketId = BucketIdentifier.partitionBucketIdStr(partitionPath, bucketNumber);
+          LOG.info(String.format("Should load this partition bucket %s with fileID %s", partitionBucketId, fileID));
+          if (bucketToFileIDMap.containsKey(partitionBucketId)) {
+            throw new RuntimeException(String.format("Duplicate fileID %s from partitionBucket %s found "
+              + "during the fileGroupPerPartitionedBucketState initialization.", fileID, partitionBucketId));
+          } else {
+            LOG.info(String.format("Adding fileID %s to the partition bucket %s.", fileID, partitionBucketId));
+            bucketToFileIDMap.put(partitionBucketId, fileID);
+          }

Review comment:
       yes, this could be store in the state, but we need to think about the consistency issue between the state and actual file system view. Any diff could lead to incorrect data.




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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] hudi-bot commented on pull request #4679: [HUDI-3315] RFC-35 Part-1 Support bucket index in Flink writer

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5471",
       "triggerID" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5491",
       "triggerID" : "1020748430",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5492",
       "triggerID" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5493",
       "triggerID" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5528",
       "triggerID" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6056",
       "triggerID" : "1041367799",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6056",
       "triggerID" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1041367799",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6189",
       "triggerID" : "1047380901",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6189",
       "triggerID" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1047380901",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6195",
       "triggerID" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6196",
       "triggerID" : "1047494529",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6195",
       "triggerID" : "1047494529",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6271",
       "triggerID" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6272",
       "triggerID" : "1049827248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6271",
       "triggerID" : "1049827248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6299",
       "triggerID" : "1050610982",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "8120f64681ac4e42a075d2cc04b036f737622ccd",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6388",
       "triggerID" : "8120f64681ac4e42a075d2cc04b036f737622ccd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8120f64681ac4e42a075d2cc04b036f737622ccd",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6390",
       "triggerID" : "1054218095",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "8120f64681ac4e42a075d2cc04b036f737622ccd",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6388",
       "triggerID" : "1054218095",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3340d3ac9a0d00b0f217a0dffc63b24960f39904",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "3340d3ac9a0d00b0f217a0dffc63b24960f39904",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 8120f64681ac4e42a075d2cc04b036f737622ccd Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6390) Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6388) 
   * 3340d3ac9a0d00b0f217a0dffc63b24960f39904 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] wxplovecc commented on pull request #4679: [HUDI-3315] RFC-35 Part-1 Support bucket index in Flink writer

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


   I have tryed and got an Exception:
   ```java
   Caused by: java.util.NoSuchElementException: No value present in Option
   	at org.apache.hudi.common.util.Option.get(Option.java:88) ~[hudi-flink-bundle_2.11-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT]
   	at org.apache.hudi.io.HoodieMergeHandle.<init>(HoodieMergeHandle.java:116) ~[hudi-flink-bundle_2.11-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT]
   	at org.apache.hudi.io.FlinkMergeHandle.<init>(FlinkMergeHandle.java:70) ~[hudi-flink-bundle_2.11-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT]
   	at org.apache.hudi.client.HoodieFlinkWriteClient.getOrCreateWriteHandle(HoodieFlinkWriteClient.java:485) ~[hudi-flink-bundle_2.11-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT]
   	at org.apache.hudi.client.HoodieFlinkWriteClient.upsert(HoodieFlinkWriteClient.java:142) ~[hudi-flink-bundle_2.11-0.11.0-SNAPSHOT.jar:0.11.0-SNAPSHOT]
   ```
   @garyli1019 
   maybe we should not set `instantTime` to `U` when the fileId is created before checkpoint
   


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] hudi-bot removed a comment on pull request #4679: [RFC-35] Make Flink writer stream friendly

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5471",
       "triggerID" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d4f6e12cca613145904db85abc3e1bd68a08e823 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5471) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] garyli1019 commented on pull request #4679: [RFC-35] Make Flink writer stream friendly

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


   @hudi-bot run azure


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] garyli1019 commented on pull request #4679: [RFC-35] Make Flink writer stream friendly

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


   @hudi-bot run azure


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] hudi-bot commented on pull request #4679: [HUDI-3315] RFC-35 Make Flink writer stream friendly

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5471",
       "triggerID" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5491",
       "triggerID" : "1020748430",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5492",
       "triggerID" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5493",
       "triggerID" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5528",
       "triggerID" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 4d49dbe71e689f0d56d3447bc5c4367db35c33af Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5528) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] danny0405 commented on a change in pull request #4679: [HUDI-3315] RFC-35 Make Flink writer stream friendly

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



##########
File path: hudi-flink/src/main/java/org/apache/hudi/configuration/FlinkOptions.java
##########
@@ -310,6 +318,20 @@ private FlinkOptions() {
           + "Actual value will be obtained by invoking .toString() on the field value. Nested fields can be specified using "
           + "the dot notation eg: `a.b.c`");
 
+  public static final ConfigOption<String> INDEX_KEY_FIELD = ConfigOptions
+      .key(HoodieIndexConfig.BUCKET_INDEX_HASH_FIELD.key())
+      .stringType()
+      .defaultValue("uuid")
+      .withDescription("Record key field. Value to be used as the `recordKey` component of `HoodieKey`.\n"
+        + "Actual value will be obtained by invoking .toString() on the field value. Nested fields can be specified using "
+        + "the dot notation eg: `a.b.c`");
+
+  public static final ConfigOption<Integer> BUCKET_INDEX_NUM_BUCKETS = ConfigOptions
+      .key(HoodieIndexConfig.BUCKET_INDEX_NUM_BUCKETS.key())
+      .intType()
+      .defaultValue(256) // default 256 buckets per partition
+      .withDescription("Hudi bucket number per partition. Only affected if using Hudi bucket index.");

Review comment:
       Is there any reason the default value is 256 here, seems to generate many small files for small data sets.




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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] hudi-bot commented on pull request #4679: [HUDI-3315] RFC-35 Make Flink writer stream friendly

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5471",
       "triggerID" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5491",
       "triggerID" : "1020748430",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5492",
       "triggerID" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5493",
       "triggerID" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5528",
       "triggerID" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "",
       "status" : "CANCELED",
       "url" : "TBD",
       "triggerID" : "1041367799",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6056",
       "triggerID" : "1041367799",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6056",
       "triggerID" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   *  Unknown: [CANCELED](TBD) 
   * ddbf502d11717d800971f736fa81054afa1873b7 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6056) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] hudi-bot removed a comment on pull request #4679: [HUDI-3315] RFC-35 Make Flink writer stream friendly

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5471",
       "triggerID" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5491",
       "triggerID" : "1020748430",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5492",
       "triggerID" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5493",
       "triggerID" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5528",
       "triggerID" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "1041367799",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "",
       "status" : "CANCELED",
       "url" : "TBD",
       "triggerID" : "1041367799",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   *  Unknown: [CANCELED](TBD) 
   * ddbf502d11717d800971f736fa81054afa1873b7 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] hudi-bot commented on pull request #4679: [HUDI-3315] RFC-35 Make Flink writer stream friendly

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5471",
       "triggerID" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5491",
       "triggerID" : "1020748430",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5492",
       "triggerID" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5493",
       "triggerID" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5528",
       "triggerID" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6056",
       "triggerID" : "1041367799",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6056",
       "triggerID" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1041367799",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "1047380901",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "",
       "status" : "CANCELED",
       "url" : "TBD",
       "triggerID" : "1047380901",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   *  Unknown: [CANCELED](TBD) 
   * 48b162c6c411b9cb5b605d006acc8d6e7d45cec8 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] hudi-bot removed a comment on pull request #4679: [RFC-35] Make Flink writer stream friendly

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d4f6e12cca613145904db85abc3e1bd68a08e823 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] hudi-bot commented on pull request #4679: [RFC-35] Make Flink writer stream friendly

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5471",
       "triggerID" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d4f6e12cca613145904db85abc3e1bd68a08e823 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5471) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] danny0405 commented on a change in pull request #4679: [HUDI-3315] RFC-35 Make Flink writer stream friendly

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



##########
File path: hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/delta/FlinkStreamUpsertDeltaCommitActionExecutor.java
##########
@@ -0,0 +1,80 @@
+/*
+ * 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.action.commit.delta;
+
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.engine.HoodieEngineContext;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.common.model.WriteOperationType;
+import org.apache.hudi.config.HoodieWriteConfig;

Review comment:
       Guess this class can be avoided if we move the buffering logic into the write function ?




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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] hudi-bot removed a comment on pull request #4679: [HUDI-3315] RFC-35 Make Flink writer stream friendly

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5471",
       "triggerID" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5491",
       "triggerID" : "1020748430",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5492",
       "triggerID" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5493",
       "triggerID" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 7810bd0c529dc44094e13650696d585d21b23c82 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5493) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] hudi-bot removed a comment on pull request #4679: [HUDI-3315] RFC-35 Make Flink writer stream friendly

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5471",
       "triggerID" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5491",
       "triggerID" : "1020748430",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5492",
       "triggerID" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5493",
       "triggerID" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5528",
       "triggerID" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 7810bd0c529dc44094e13650696d585d21b23c82 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5493) 
   * 4d49dbe71e689f0d56d3447bc5c4367db35c33af Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5528) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] garyli1019 commented on pull request #4679: [HUDI-3315] RFC-35 Part-1 Support bucket index in Flink writer

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


   @hudi-bot run azure


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] hudi-bot commented on pull request #4679: [HUDI-3315] RFC-35 Part-1 Support bucket index in Flink writer

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5471",
       "triggerID" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5491",
       "triggerID" : "1020748430",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5492",
       "triggerID" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5493",
       "triggerID" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5528",
       "triggerID" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6056",
       "triggerID" : "1041367799",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6056",
       "triggerID" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1041367799",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6189",
       "triggerID" : "1047380901",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6189",
       "triggerID" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1047380901",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6195",
       "triggerID" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6196",
       "triggerID" : "1047494529",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6195",
       "triggerID" : "1047494529",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6271",
       "triggerID" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6272",
       "triggerID" : "1049827248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6271",
       "triggerID" : "1049827248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6299",
       "triggerID" : "1050610982",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * 96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6272) Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6271) Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6299) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] hudi-bot removed a comment on pull request #4679: [HUDI-3315] RFC-35 Part-1 Support bucket index in Flink writer

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5471",
       "triggerID" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5491",
       "triggerID" : "1020748430",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5492",
       "triggerID" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5493",
       "triggerID" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5528",
       "triggerID" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6056",
       "triggerID" : "1041367799",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6056",
       "triggerID" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1041367799",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6189",
       "triggerID" : "1047380901",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6189",
       "triggerID" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1047380901",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6195",
       "triggerID" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6196",
       "triggerID" : "1047494529",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6195",
       "triggerID" : "1047494529",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6271",
       "triggerID" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6272",
       "triggerID" : "1049827248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6271",
       "triggerID" : "1049827248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6299",
       "triggerID" : "1050610982",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "8120f64681ac4e42a075d2cc04b036f737622ccd",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6388",
       "triggerID" : "8120f64681ac4e42a075d2cc04b036f737622ccd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8120f64681ac4e42a075d2cc04b036f737622ccd",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6390",
       "triggerID" : "1054218095",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * 96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6272) Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6271) Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6299) 
   * 8120f64681ac4e42a075d2cc04b036f737622ccd Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6388) Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6390) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] hudi-bot commented on pull request #4679: [HUDI-3315] RFC-35 Part-1 Support bucket index in Flink writer

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5471",
       "triggerID" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5491",
       "triggerID" : "1020748430",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5492",
       "triggerID" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5493",
       "triggerID" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5528",
       "triggerID" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6056",
       "triggerID" : "1041367799",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6056",
       "triggerID" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1041367799",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6189",
       "triggerID" : "1047380901",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6189",
       "triggerID" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1047380901",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6195",
       "triggerID" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6196",
       "triggerID" : "1047494529",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6195",
       "triggerID" : "1047494529",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6271",
       "triggerID" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6272",
       "triggerID" : "1049827248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6271",
       "triggerID" : "1049827248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6299",
       "triggerID" : "1050610982",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "8120f64681ac4e42a075d2cc04b036f737622ccd",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6388",
       "triggerID" : "8120f64681ac4e42a075d2cc04b036f737622ccd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8120f64681ac4e42a075d2cc04b036f737622ccd",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6390",
       "triggerID" : "1054218095",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * 96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6272) Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6271) Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6299) 
   * 8120f64681ac4e42a075d2cc04b036f737622ccd Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6388) Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6390) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] hudi-bot commented on pull request #4679: [HUDI-3315] RFC-35 Part-1 Support bucket index in Flink writer

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5471",
       "triggerID" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5491",
       "triggerID" : "1020748430",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5492",
       "triggerID" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5493",
       "triggerID" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5528",
       "triggerID" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6056",
       "triggerID" : "1041367799",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6056",
       "triggerID" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1041367799",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6189",
       "triggerID" : "1047380901",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6189",
       "triggerID" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1047380901",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6195",
       "triggerID" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6196",
       "triggerID" : "1047494529",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6195",
       "triggerID" : "1047494529",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6271",
       "triggerID" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6272",
       "triggerID" : "1049827248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6271",
       "triggerID" : "1049827248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6299",
       "triggerID" : "1050610982",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "8120f64681ac4e42a075d2cc04b036f737622ccd",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6388",
       "triggerID" : "8120f64681ac4e42a075d2cc04b036f737622ccd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8120f64681ac4e42a075d2cc04b036f737622ccd",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6390",
       "triggerID" : "1054218095",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "8120f64681ac4e42a075d2cc04b036f737622ccd",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6388",
       "triggerID" : "1054218095",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * 8120f64681ac4e42a075d2cc04b036f737622ccd Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6390) Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6388) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] hudi-bot commented on pull request #4679: [HUDI-3315] RFC-35 Part-1 Support bucket index in Flink writer

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5471",
       "triggerID" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5491",
       "triggerID" : "1020748430",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5492",
       "triggerID" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5493",
       "triggerID" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5528",
       "triggerID" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6056",
       "triggerID" : "1041367799",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6056",
       "triggerID" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1041367799",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6189",
       "triggerID" : "1047380901",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6189",
       "triggerID" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1047380901",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6195",
       "triggerID" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6196",
       "triggerID" : "1047494529",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6195",
       "triggerID" : "1047494529",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6271",
       "triggerID" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6272",
       "triggerID" : "1049827248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6271",
       "triggerID" : "1049827248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6299",
       "triggerID" : "1050610982",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "8120f64681ac4e42a075d2cc04b036f737622ccd",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6388",
       "triggerID" : "8120f64681ac4e42a075d2cc04b036f737622ccd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8120f64681ac4e42a075d2cc04b036f737622ccd",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6390",
       "triggerID" : "1054218095",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "8120f64681ac4e42a075d2cc04b036f737622ccd",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6388",
       "triggerID" : "1054218095",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3340d3ac9a0d00b0f217a0dffc63b24960f39904",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6429",
       "triggerID" : "3340d3ac9a0d00b0f217a0dffc63b24960f39904",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 8120f64681ac4e42a075d2cc04b036f737622ccd Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6390) Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6388) 
   * 3340d3ac9a0d00b0f217a0dffc63b24960f39904 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6429) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] hudi-bot commented on pull request #4679: [HUDI-3315] RFC-35 Part-1 Support bucket index in Flink writer

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5471",
       "triggerID" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5491",
       "triggerID" : "1020748430",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5492",
       "triggerID" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5493",
       "triggerID" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5528",
       "triggerID" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6056",
       "triggerID" : "1041367799",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6056",
       "triggerID" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1041367799",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6189",
       "triggerID" : "1047380901",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6189",
       "triggerID" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1047380901",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6195",
       "triggerID" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6196",
       "triggerID" : "1047494529",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6195",
       "triggerID" : "1047494529",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6271",
       "triggerID" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6272",
       "triggerID" : "1049827248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6271",
       "triggerID" : "1049827248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6299",
       "triggerID" : "1050610982",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "8120f64681ac4e42a075d2cc04b036f737622ccd",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6388",
       "triggerID" : "8120f64681ac4e42a075d2cc04b036f737622ccd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8120f64681ac4e42a075d2cc04b036f737622ccd",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6390",
       "triggerID" : "1054218095",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "8120f64681ac4e42a075d2cc04b036f737622ccd",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6388",
       "triggerID" : "1054218095",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * 8120f64681ac4e42a075d2cc04b036f737622ccd Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6390) Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6388) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] danny0405 commented on a change in pull request #4679: [HUDI-3315] RFC-35 Part-1 Support bucket index in Flink writer

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



##########
File path: hudi-flink/src/main/java/org/apache/hudi/sink/BucketStreamWriteFunction.java
##########
@@ -0,0 +1,158 @@
+/*
+ * 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.configuration.Configuration;
+import org.apache.flink.runtime.state.FunctionInitializationContext;
+import org.apache.flink.streaming.api.functions.ProcessFunction;
+import org.apache.flink.util.Collector;
+import org.apache.hudi.common.model.FileSlice;
+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.table.timeline.HoodieInstant;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.configuration.FlinkOptions;
+import org.apache.hudi.index.bucket.BucketIdentifier;
+import org.apache.hudi.table.HoodieFlinkTable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+
+import static java.util.stream.Collectors.toList;
+
+public class BucketStreamWriteFunction<I> extends StreamWriteFunction<I> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BucketStreamWriteFunction.class);
+
+  private int taskTotalParallelism;
+
+  private int currentParallelism;
+
+  private int hiveBucketNum;
+
+  protected transient HoodieFlinkTable table;
+
+  private String indexKeyFields;
+
+  private HashMap<String, String> bucketToFileIDMap;
+
+  /**
+   * Constructs a BucketStreamWriteFunction.
+   *
+   * @param config The config options
+   */
+  public BucketStreamWriteFunction(Configuration config) {
+    super(config);
+    this.bucketToFileIDMap = new HashMap<String, String>();
+  }
+
+  @Override
+  public void open(Configuration parameters) throws IOException {
+    super.open(parameters);
+    this.hiveBucketNum = config.getInteger(FlinkOptions.BUCKET_INDEX_NUM_BUCKETS);
+    this.indexKeyFields = config.getString(FlinkOptions.INDEX_KEY_FIELD);
+    this.taskID = getRuntimeContext().getIndexOfThisSubtask();
+    this.currentParallelism = getRuntimeContext().getNumberOfParallelSubtasks();
+    this.taskTotalParallelism = getRuntimeContext().getMaxNumberOfParallelSubtasks();
+    bootstrapIndex();
+  }
+
+  @Override
+  public void initializeState(FunctionInitializationContext context) throws Exception {
+    super.initializeState(context);
+    this.table = this.writeClient.getHoodieTable();
+  }
+
+  @Override
+  public void processElement(I i, ProcessFunction<I, Object>.Context context, Collector<Object> collector) throws Exception {
+    HoodieRecord<?> record = (HoodieRecord<?>) i;
+    final HoodieKey hoodieKey = record.getKey();
+    final HoodieRecordLocation location;
+
+    final int bucketNum = BucketIdentifier.getBucketId(hoodieKey, indexKeyFields, hiveBucketNum);
+    final String partitionBucketId = BucketIdentifier.partitionBucketIdStr(hoodieKey.getPartitionPath(), bucketNum);
+
+    if (bucketToFileIDMap.containsKey(partitionBucketId)) {
+      location = new HoodieRecordLocation("U", bucketToFileIDMap.get(partitionBucketId));
+    } else {
+      String newFileId = BucketIdentifier.newBucketFileIdPrefix(bucketNum);
+      location = new HoodieRecordLocation("I", newFileId);
+      bucketToFileIDMap.put(partitionBucketId, newFileId);
+    }
+    record.unseal();
+    record.setCurrentLocation(location);
+    record.seal();
+    bufferRecord(record);
+  }
+
+  /**
+   * Get partition_bucket -> fileID mapping from the existing hudi table.
+   * This is a required operation for each restart to avoid having duplicate file ids for one bucket.
+   */
+  private void bootstrapIndex() throws IOException {
+    Option<HoodieInstant> latestCommitTime = table.getFileSystemView().getTimeline().filterCompletedInstants().lastInstant();
+    if (!latestCommitTime.isPresent()) {
+      return;
+    }
+    // bootstrap bucket info from existing file system
+    // bucketNum % totalParallelism == this taskID belongs to this task
+    HashSet<Integer> bucketToLoad = new HashSet<>();
+    for (int i = 0; i < hiveBucketNum; i++) {
+      int partitionOfBucket = BucketIdentifier.mod(i, currentParallelism);
+      if (partitionOfBucket == taskID) {
+        LOG.info(String.format("Bootstrapping index. Adding bucket %s , "
+            + "Current parallelism: %s , Max parallelism: %s , Current task id: %s",
+            i, currentParallelism, taskTotalParallelism, taskID));
+        bucketToLoad.add(i);
+      }
+    }
+    bucketToLoad.stream().forEach(bucket -> LOG.info(String.format("bucketToLoad contains %s", bucket)));
+
+    LOG.info(String.format("Loading Hoodie Table %s, with path %s", table.getMetaClient().getTableConfig().getTableName(),
+        table.getMetaClient().getBasePath()));
+
+    // Iterate through all existing partitions to load existing fileID belongs to this task
+    List<String> partitions = table.getMetadata().getAllPartitionPaths();
+    for (String partitionPath : partitions) {
+      List<FileSlice> latestFileSlices = table.getSliceView()
+          .getLatestFileSlices(partitionPath)
+          .collect(toList());
+      for (FileSlice fileslice : latestFileSlices) {
+        String fileID = fileslice.getFileId();
+        int bucketNumber = BucketIdentifier.bucketIdFromFileId(fileID);
+        if (bucketToLoad.contains(bucketNumber)) {
+          String partitionBucketId = BucketIdentifier.partitionBucketIdStr(partitionPath, bucketNumber);
+          LOG.info(String.format("Should load this partition bucket %s with fileID %s", partitionBucketId, fileID));
+          if (bucketToFileIDMap.containsKey(partitionBucketId)) {
+            throw new RuntimeException(String.format("Duplicate fileID %s from partitionBucket %s found "
+              + "during the fileGroupPerPartitionedBucketState initialization.", fileID, partitionBucketId));
+          } else {
+            LOG.info(String.format("Adding fileID %s to the partition bucket %s.", fileID, partitionBucketId));
+            bucketToFileIDMap.put(partitionBucketId, fileID);
+          }

Review comment:
       The checkpoint can keep the correctness i think.




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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] danny0405 commented on a change in pull request #4679: [HUDI-3315] RFC-35 Part-1 Support bucket index in Flink writer

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



##########
File path: hudi-flink/src/main/java/org/apache/hudi/configuration/FlinkOptions.java
##########
@@ -310,6 +318,20 @@ private FlinkOptions() {
           + "Actual value will be obtained by invoking .toString() on the field value. Nested fields can be specified using "
           + "the dot notation eg: `a.b.c`");
 
+  public static final ConfigOption<String> INDEX_KEY_FIELD = ConfigOptions
+      .key(HoodieIndexConfig.BUCKET_INDEX_HASH_FIELD.key())
+      .stringType()
+      .defaultValue("uuid")

Review comment:
       > If the index key field not set, use the record key field as default to make the configuration more user-friendly?
   
   A good suggestion, i would prefer this way so that user do not need to declare the keys twice. The bucket number is a little different i think and it can have a default value.




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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] garyli1019 commented on pull request #4679: [HUDI-3315] RFC-35 Make Flink writer stream friendly

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


   As discussed with @danny0405 , the changes in write client will be not included in this PR, because it will make the write client looks ugly. We will include those once we have a streaming API. So this PR will only include the bucket index for Flink writer. cc: @yihua 


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] hudi-bot removed a comment on pull request #4679: [HUDI-3315] RFC-35 Make Flink writer stream friendly

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5471",
       "triggerID" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5491",
       "triggerID" : "1020748430",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5492",
       "triggerID" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5493",
       "triggerID" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5528",
       "triggerID" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6056",
       "triggerID" : "1041367799",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6056",
       "triggerID" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1041367799",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "",
       "status" : "CANCELED",
       "url" : "TBD",
       "triggerID" : "1047380901",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6189",
       "triggerID" : "1047380901",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6189",
       "triggerID" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   *  Unknown: [CANCELED](TBD) 
   * 48b162c6c411b9cb5b605d006acc8d6e7d45cec8 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6189) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] garyli1019 commented on pull request #4679: [HUDI-3315] RFC-35 Make Flink writer stream friendly

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


   @hudi-bot run azure


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] hudi-bot removed a comment on pull request #4679: [HUDI-3315] RFC-35 Make Flink writer stream friendly

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5471",
       "triggerID" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5491",
       "triggerID" : "1020748430",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5492",
       "triggerID" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5493",
       "triggerID" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5528",
       "triggerID" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6056",
       "triggerID" : "1041367799",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6056",
       "triggerID" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1041367799",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6189",
       "triggerID" : "1047380901",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6189",
       "triggerID" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1047380901",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6195",
       "triggerID" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6196",
       "triggerID" : "1047494529",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * 48b162c6c411b9cb5b605d006acc8d6e7d45cec8 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6189) 
   * 2e89b137fc1ede6cc5bd95fdf2e414eea706b091 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6195) Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6196) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] mincwang commented on pull request #4679: [HUDI-3315] RFC-35 Part-1 Support bucket index in Flink writer

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


   🆒  I will try to this the PR soon. thanks for the bigolds. 🙏


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] danny0405 commented on a change in pull request #4679: [HUDI-3315] RFC-35 Part-1 Support bucket index in Flink writer

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



##########
File path: hudi-flink/src/main/java/org/apache/hudi/configuration/FlinkOptions.java
##########
@@ -106,6 +108,12 @@ private FlinkOptions() {
   // ------------------------------------------------------------------------
   //  Index Options
   // ------------------------------------------------------------------------
+  public static final ConfigOption<String> INDEX_TYPE = ConfigOptions
+      .key("write.index.type")
+      .stringType()

Review comment:
       `write.index.type` => `index.type` ?

##########
File path: hudi-flink/src/main/java/org/apache/hudi/configuration/FlinkOptions.java
##########
@@ -310,6 +318,20 @@ private FlinkOptions() {
           + "Actual value will be obtained by invoking .toString() on the field value. Nested fields can be specified using "
           + "the dot notation eg: `a.b.c`");
 
+  public static final ConfigOption<String> INDEX_KEY_FIELD = ConfigOptions
+      .key(HoodieIndexConfig.BUCKET_INDEX_HASH_FIELD.key())
+      .stringType()
+      .defaultValue("uuid")
+      .withDescription("Record key field. Value to be used as the `recordKey` component of `HoodieKey`.\n"
+        + "Actual value will be obtained by invoking .toString() on the field value. Nested fields can be specified using "
+        + "the dot notation eg: `a.b.c`");
+
+  public static final ConfigOption<Integer> BUCKET_INDEX_NUM_BUCKETS = ConfigOptions
+      .key(HoodieIndexConfig.BUCKET_INDEX_NUM_BUCKETS.key())
+      .intType()
+      .defaultValue(256) // default 256 buckets per partition
+      .withDescription("Hudi bucket number per partition. Only affected if using Hudi bucket index.");

Review comment:
       If people change the `BUCKET_INDEX_NUM_BUCKETS` or the write function parallelism, does the hash index still work ?

##########
File path: hudi-flink/src/main/java/org/apache/hudi/sink/BucketStreamWriteFunction.java
##########
@@ -0,0 +1,158 @@
+/*
+ * 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.configuration.Configuration;
+import org.apache.flink.runtime.state.FunctionInitializationContext;
+import org.apache.flink.streaming.api.functions.ProcessFunction;
+import org.apache.flink.util.Collector;
+import org.apache.hudi.common.model.FileSlice;
+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.table.timeline.HoodieInstant;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.configuration.FlinkOptions;
+import org.apache.hudi.index.bucket.BucketIdentifier;
+import org.apache.hudi.table.HoodieFlinkTable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+
+import static java.util.stream.Collectors.toList;
+
+public class BucketStreamWriteFunction<I> extends StreamWriteFunction<I> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BucketStreamWriteFunction.class);
+
+  private int taskTotalParallelism;
+
+  private int currentParallelism;
+
+  private int hiveBucketNum;
+
+  protected transient HoodieFlinkTable table;
+
+  private String indexKeyFields;
+
+  private HashMap<String, String> bucketToFileIDMap;
+
+  /**
+   * Constructs a BucketStreamWriteFunction.
+   *
+   * @param config The config options
+   */
+  public BucketStreamWriteFunction(Configuration config) {
+    super(config);
+    this.bucketToFileIDMap = new HashMap<String, String>();
+  }
+
+  @Override
+  public void open(Configuration parameters) throws IOException {
+    super.open(parameters);
+    this.hiveBucketNum = config.getInteger(FlinkOptions.BUCKET_INDEX_NUM_BUCKETS);
+    this.indexKeyFields = config.getString(FlinkOptions.INDEX_KEY_FIELD);
+    this.taskID = getRuntimeContext().getIndexOfThisSubtask();
+    this.currentParallelism = getRuntimeContext().getNumberOfParallelSubtasks();
+    this.taskTotalParallelism = getRuntimeContext().getMaxNumberOfParallelSubtasks();
+    bootstrapIndex();
+  }
+
+  @Override
+  public void initializeState(FunctionInitializationContext context) throws Exception {
+    super.initializeState(context);
+    this.table = this.writeClient.getHoodieTable();
+  }
+
+  @Override
+  public void processElement(I i, ProcessFunction<I, Object>.Context context, Collector<Object> collector) throws Exception {
+    HoodieRecord<?> record = (HoodieRecord<?>) i;
+    final HoodieKey hoodieKey = record.getKey();
+    final HoodieRecordLocation location;
+
+    final int bucketNum = BucketIdentifier.getBucketId(hoodieKey, indexKeyFields, hiveBucketNum);
+    final String partitionBucketId = BucketIdentifier.partitionBucketIdStr(hoodieKey.getPartitionPath(), bucketNum);
+
+    if (bucketToFileIDMap.containsKey(partitionBucketId)) {
+      location = new HoodieRecordLocation("U", bucketToFileIDMap.get(partitionBucketId));
+    } else {
+      String newFileId = BucketIdentifier.newBucketFileIdPrefix(bucketNum);
+      location = new HoodieRecordLocation("I", newFileId);
+      bucketToFileIDMap.put(partitionBucketId, newFileId);
+    }
+    record.unseal();
+    record.setCurrentLocation(location);
+    record.seal();
+    bufferRecord(record);
+  }
+
+  /**
+   * Get partition_bucket -> fileID mapping from the existing hudi table.
+   * This is a required operation for each restart to avoid having duplicate file ids for one bucket.
+   */
+  private void bootstrapIndex() throws IOException {
+    Option<HoodieInstant> latestCommitTime = table.getFileSystemView().getTimeline().filterCompletedInstants().lastInstant();
+    if (!latestCommitTime.isPresent()) {
+      return;
+    }
+    // bootstrap bucket info from existing file system
+    // bucketNum % totalParallelism == this taskID belongs to this task
+    HashSet<Integer> bucketToLoad = new HashSet<>();
+    for (int i = 0; i < hiveBucketNum; i++) {
+      int partitionOfBucket = BucketIdentifier.mod(i, currentParallelism);
+      if (partitionOfBucket == taskID) {
+        LOG.info(String.format("Bootstrapping index. Adding bucket %s , "
+            + "Current parallelism: %s , Max parallelism: %s , Current task id: %s",
+            i, currentParallelism, taskTotalParallelism, taskID));
+        bucketToLoad.add(i);
+      }
+    }
+    bucketToLoad.stream().forEach(bucket -> LOG.info(String.format("bucketToLoad contains %s", bucket)));
+
+    LOG.info(String.format("Loading Hoodie Table %s, with path %s", table.getMetaClient().getTableConfig().getTableName(),
+        table.getMetaClient().getBasePath()));
+
+    // Iterate through all existing partitions to load existing fileID belongs to this task
+    List<String> partitions = table.getMetadata().getAllPartitionPaths();
+    for (String partitionPath : partitions) {
+      List<FileSlice> latestFileSlices = table.getSliceView()
+          .getLatestFileSlices(partitionPath)
+          .collect(toList());
+      for (FileSlice fileslice : latestFileSlices) {
+        String fileID = fileslice.getFileId();
+        int bucketNumber = BucketIdentifier.bucketIdFromFileId(fileID);
+        if (bucketToLoad.contains(bucketNumber)) {
+          String partitionBucketId = BucketIdentifier.partitionBucketIdStr(partitionPath, bucketNumber);
+          LOG.info(String.format("Should load this partition bucket %s with fileID %s", partitionBucketId, fileID));
+          if (bucketToFileIDMap.containsKey(partitionBucketId)) {
+            throw new RuntimeException(String.format("Duplicate fileID %s from partitionBucket %s found "
+              + "during the fileGroupPerPartitionedBucketState initialization.", fileID, partitionBucketId));
+          } else {
+            LOG.info(String.format("Adding fileID %s to the partition bucket %s.", fileID, partitionBucketId));
+            bucketToFileIDMap.put(partitionBucketId, fileID);
+          }

Review comment:
       The bucketToFileIDMap seems never be cleared, is there possibility that this map be put into the state ?

##########
File path: hudi-flink/src/main/java/org/apache/hudi/configuration/OptionsResolver.java
##########
@@ -101,6 +102,10 @@ public static boolean isPartitionedTable(Configuration conf) {
     return FilePathUtils.extractPartitionKeys(conf).length > 0;
   }
 
+  public static boolean isBucketIndexTable(Configuration conf) {
+    return conf.getString(FlinkOptions.INDEX_TYPE).equals(HoodieIndex.IndexType.BUCKET.name());

Review comment:
       `isBucketIndexTable` => `isBucketIndexType`

##########
File path: hudi-flink/src/main/java/org/apache/hudi/sink/BucketStreamWriteFunction.java
##########
@@ -0,0 +1,158 @@
+/*
+ * 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.configuration.Configuration;
+import org.apache.flink.runtime.state.FunctionInitializationContext;
+import org.apache.flink.streaming.api.functions.ProcessFunction;
+import org.apache.flink.util.Collector;
+import org.apache.hudi.common.model.FileSlice;
+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.table.timeline.HoodieInstant;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.configuration.FlinkOptions;
+import org.apache.hudi.index.bucket.BucketIdentifier;
+import org.apache.hudi.table.HoodieFlinkTable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+
+import static java.util.stream.Collectors.toList;
+
+public class BucketStreamWriteFunction<I> extends StreamWriteFunction<I> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BucketStreamWriteFunction.class);
+
+  private int taskTotalParallelism;
+
+  private int currentParallelism;
+
+  private int hiveBucketNum;
+
+  protected transient HoodieFlinkTable table;
+
+  private String indexKeyFields;
+
+  private HashMap<String, String> bucketToFileIDMap;
+
+  /**
+   * Constructs a BucketStreamWriteFunction.
+   *
+   * @param config The config options
+   */
+  public BucketStreamWriteFunction(Configuration config) {
+    super(config);
+    this.bucketToFileIDMap = new HashMap<String, String>();
+  }
+
+  @Override
+  public void open(Configuration parameters) throws IOException {
+    super.open(parameters);
+    this.hiveBucketNum = config.getInteger(FlinkOptions.BUCKET_INDEX_NUM_BUCKETS);
+    this.indexKeyFields = config.getString(FlinkOptions.INDEX_KEY_FIELD);
+    this.taskID = getRuntimeContext().getIndexOfThisSubtask();
+    this.currentParallelism = getRuntimeContext().getNumberOfParallelSubtasks();
+    this.taskTotalParallelism = getRuntimeContext().getMaxNumberOfParallelSubtasks();
+    bootstrapIndex();

Review comment:
       `this.currentParallelism` => `this.parallelism`
   `this.taskTotalParallelism` => `this.maxParallelism`

##########
File path: hudi-flink/src/main/java/org/apache/hudi/configuration/FlinkOptions.java
##########
@@ -310,6 +318,20 @@ private FlinkOptions() {
           + "Actual value will be obtained by invoking .toString() on the field value. Nested fields can be specified using "
           + "the dot notation eg: `a.b.c`");
 
+  public static final ConfigOption<String> INDEX_KEY_FIELD = ConfigOptions
+      .key(HoodieIndexConfig.BUCKET_INDEX_HASH_FIELD.key())
+      .stringType()

Review comment:
       Does this key must be same same with the primary key ? Because all the changes of a key must belong to one data bucket.

##########
File path: hudi-flink/src/main/java/org/apache/hudi/sink/common/AbstractStreamWriteFunction.java
##########
@@ -107,7 +107,7 @@
   /**
    * List state of the write metadata events.
    */
-  private transient ListState<WriteMetadataEvent> writeMetadataState;
+  protected transient ListState<WriteMetadataEvent> writeMetadataState;
 

Review comment:
       Seems not used by sub-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.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] hudi-bot commented on pull request #4679: [HUDI-3315] RFC-35 Part-1 Support bucket index in Flink writer

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5471",
       "triggerID" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5491",
       "triggerID" : "1020748430",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5492",
       "triggerID" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5493",
       "triggerID" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5528",
       "triggerID" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6056",
       "triggerID" : "1041367799",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6056",
       "triggerID" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1041367799",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6189",
       "triggerID" : "1047380901",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6189",
       "triggerID" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1047380901",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6195",
       "triggerID" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6196",
       "triggerID" : "1047494529",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6195",
       "triggerID" : "1047494529",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6271",
       "triggerID" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6272",
       "triggerID" : "1049827248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6271",
       "triggerID" : "1049827248",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * 96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6272) Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6271) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] hudi-bot removed a comment on pull request #4679: [HUDI-3315] RFC-35 Part-1 Support bucket index in Flink writer

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5471",
       "triggerID" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5491",
       "triggerID" : "1020748430",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5492",
       "triggerID" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5493",
       "triggerID" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5528",
       "triggerID" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6056",
       "triggerID" : "1041367799",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6056",
       "triggerID" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1041367799",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6189",
       "triggerID" : "1047380901",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6189",
       "triggerID" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1047380901",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6195",
       "triggerID" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6196",
       "triggerID" : "1047494529",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6195",
       "triggerID" : "1047494529",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6271",
       "triggerID" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6272",
       "triggerID" : "1049827248",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * 2e89b137fc1ede6cc5bd95fdf2e414eea706b091 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6196) Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6195) 
   * 96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6271) Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6272) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] hudi-bot commented on pull request #4679: [HUDI-3315] RFC-35 Part-1 Support bucket index in Flink writer

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5471",
       "triggerID" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5491",
       "triggerID" : "1020748430",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5492",
       "triggerID" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5493",
       "triggerID" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5528",
       "triggerID" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6056",
       "triggerID" : "1041367799",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6056",
       "triggerID" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1041367799",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6189",
       "triggerID" : "1047380901",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6189",
       "triggerID" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1047380901",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6195",
       "triggerID" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6196",
       "triggerID" : "1047494529",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6195",
       "triggerID" : "1047494529",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6271",
       "triggerID" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6272",
       "triggerID" : "1049827248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6271",
       "triggerID" : "1049827248",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * 96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6272) Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6271) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] garyli1019 commented on a change in pull request #4679: [HUDI-3315] RFC-35 Part-1 Support bucket index in Flink writer

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



##########
File path: hudi-flink/src/main/java/org/apache/hudi/sink/BucketStreamWriteFunction.java
##########
@@ -0,0 +1,158 @@
+/*
+ * 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.configuration.Configuration;
+import org.apache.flink.runtime.state.FunctionInitializationContext;
+import org.apache.flink.streaming.api.functions.ProcessFunction;
+import org.apache.flink.util.Collector;
+import org.apache.hudi.common.model.FileSlice;
+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.table.timeline.HoodieInstant;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.configuration.FlinkOptions;
+import org.apache.hudi.index.bucket.BucketIdentifier;
+import org.apache.hudi.table.HoodieFlinkTable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+
+import static java.util.stream.Collectors.toList;
+
+public class BucketStreamWriteFunction<I> extends StreamWriteFunction<I> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BucketStreamWriteFunction.class);
+
+  private int maxParallelism;
+
+  private int parallelism;
+
+  private int hiveBucketNum;
+
+  protected transient HoodieFlinkTable table;
+
+  private String indexKeyFields;
+
+  private HashMap<String, String> bucketToFileIDMap;
+
+  /**
+   * Constructs a BucketStreamWriteFunction.
+   *
+   * @param config The config options
+   */
+  public BucketStreamWriteFunction(Configuration config) {
+    super(config);
+    this.bucketToFileIDMap = new HashMap<String, String>();
+  }
+
+  @Override
+  public void open(Configuration parameters) throws IOException {
+    super.open(parameters);
+    this.hiveBucketNum = config.getInteger(FlinkOptions.BUCKET_INDEX_NUM_BUCKETS);
+    this.indexKeyFields = config.getString(FlinkOptions.INDEX_KEY_FIELD);
+    this.taskID = getRuntimeContext().getIndexOfThisSubtask();
+    this.parallelism = getRuntimeContext().getNumberOfParallelSubtasks();
+    this.maxParallelism = getRuntimeContext().getMaxNumberOfParallelSubtasks();
+    bootstrapIndex();
+  }
+
+  @Override
+  public void initializeState(FunctionInitializationContext context) throws Exception {
+    super.initializeState(context);
+    this.table = this.writeClient.getHoodieTable();
+  }
+
+  @Override
+  public void processElement(I i, ProcessFunction<I, Object>.Context context, Collector<Object> collector) throws Exception {
+    HoodieRecord<?> record = (HoodieRecord<?>) i;
+    final HoodieKey hoodieKey = record.getKey();
+    final HoodieRecordLocation location;
+
+    final int bucketNum = BucketIdentifier.getBucketId(hoodieKey, indexKeyFields, hiveBucketNum);
+    final String partitionBucketId = BucketIdentifier.partitionBucketIdStr(hoodieKey.getPartitionPath(), bucketNum);
+
+    if (bucketToFileIDMap.containsKey(partitionBucketId)) {
+      location = new HoodieRecordLocation("U", bucketToFileIDMap.get(partitionBucketId));
+    } else {
+      String newFileId = BucketIdentifier.newBucketFileIdPrefix(bucketNum);
+      location = new HoodieRecordLocation("I", newFileId);
+      bucketToFileIDMap.put(partitionBucketId, newFileId);
+    }
+    record.unseal();
+    record.setCurrentLocation(location);
+    record.seal();
+    bufferRecord(record);
+  }
+
+  /**
+   * Get partition_bucket -> fileID mapping from the existing hudi table.
+   * This is a required operation for each restart to avoid having duplicate file ids for one bucket.
+   */
+  private void bootstrapIndex() throws IOException {
+    Option<HoodieInstant> latestCommitTime = table.getFileSystemView().getTimeline().filterCompletedInstants().lastInstant();
+    if (!latestCommitTime.isPresent()) {
+      return;
+    }
+    // bootstrap bucket info from existing file system
+    // bucketNum % totalParallelism == this taskID belongs to this task
+    HashSet<Integer> bucketToLoad = new HashSet<>();
+    for (int i = 0; i < hiveBucketNum; i++) {
+      int partitionOfBucket = BucketIdentifier.mod(i, parallelism);
+      if (partitionOfBucket == taskID) {
+        LOG.info(String.format("Bootstrapping index. Adding bucket %s , "
+            + "Current parallelism: %s , Max parallelism: %s , Current task id: %s",
+            i, parallelism, maxParallelism, taskID));
+        bucketToLoad.add(i);
+      }
+    }
+    bucketToLoad.stream().forEach(bucket -> LOG.info(String.format("bucketToLoad contains %s", bucket)));
+
+    LOG.info(String.format("Loading Hoodie Table %s, with path %s", table.getMetaClient().getTableConfig().getTableName(),
+        table.getMetaClient().getBasePath()));
+
+    // Iterate through all existing partitions to load existing fileID belongs to this task
+    List<String> partitions = table.getMetadata().getAllPartitionPaths();

Review comment:
       let's do the optimization part in a separate 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.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] hudi-bot removed a comment on pull request #4679: [HUDI-3315] RFC-35 Part-1 Support bucket index in Flink writer

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5471",
       "triggerID" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5491",
       "triggerID" : "1020748430",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5492",
       "triggerID" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5493",
       "triggerID" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5528",
       "triggerID" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6056",
       "triggerID" : "1041367799",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6056",
       "triggerID" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1041367799",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6189",
       "triggerID" : "1047380901",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6189",
       "triggerID" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1047380901",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6195",
       "triggerID" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6196",
       "triggerID" : "1047494529",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6195",
       "triggerID" : "1047494529",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6271",
       "triggerID" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6272",
       "triggerID" : "1049827248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6271",
       "triggerID" : "1049827248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6299",
       "triggerID" : "1050610982",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "8120f64681ac4e42a075d2cc04b036f737622ccd",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6388",
       "triggerID" : "8120f64681ac4e42a075d2cc04b036f737622ccd",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6272) Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6271) Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6299) 
   * 8120f64681ac4e42a075d2cc04b036f737622ccd Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6388) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] hudi-bot removed a comment on pull request #4679: [HUDI-3315] RFC-35 Make Flink writer stream friendly

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5471",
       "triggerID" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5491",
       "triggerID" : "1020748430",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5492",
       "triggerID" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d4f6e12cca613145904db85abc3e1bd68a08e823 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5471) Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5491) 
   * 919d42914185e265b21d6124090bbda7a882f0f9 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5492) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] hudi-bot commented on pull request #4679: [RFC-35] Make Flink writer stream friendly

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d4f6e12cca613145904db85abc3e1bd68a08e823 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] garyli1019 commented on pull request #4679: [RFC-35] Make Flink writer stream friendly

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


   cc: @yihua FYI


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] hudi-bot removed a comment on pull request #4679: [HUDI-3315] RFC-35 Part-1 Support bucket index in Flink writer

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5471",
       "triggerID" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5491",
       "triggerID" : "1020748430",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5492",
       "triggerID" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5493",
       "triggerID" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5528",
       "triggerID" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6056",
       "triggerID" : "1041367799",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6056",
       "triggerID" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1041367799",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6189",
       "triggerID" : "1047380901",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6189",
       "triggerID" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1047380901",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6195",
       "triggerID" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6196",
       "triggerID" : "1047494529",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6195",
       "triggerID" : "1047494529",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6271",
       "triggerID" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6272",
       "triggerID" : "1049827248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6271",
       "triggerID" : "1049827248",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * 96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6272) Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6271) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] garyli1019 commented on pull request #4679: [HUDI-3315] RFC-35 Make Flink writer stream friendly

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


   @minihippo would you review this PR if you have time. 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.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] hudi-bot commented on pull request #4679: [HUDI-3315] RFC-35 Part-1 Support bucket index in Flink writer

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5471",
       "triggerID" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5491",
       "triggerID" : "1020748430",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5492",
       "triggerID" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5493",
       "triggerID" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5528",
       "triggerID" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6056",
       "triggerID" : "1041367799",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6056",
       "triggerID" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1041367799",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6189",
       "triggerID" : "1047380901",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6189",
       "triggerID" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1047380901",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6195",
       "triggerID" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6196",
       "triggerID" : "1047494529",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6195",
       "triggerID" : "1047494529",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * 2e89b137fc1ede6cc5bd95fdf2e414eea706b091 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6196) Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6195) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] hudi-bot commented on pull request #4679: [HUDI-3315] RFC-35 Make Flink writer stream friendly

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5471",
       "triggerID" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5491",
       "triggerID" : "1020748430",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5492",
       "triggerID" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5493",
       "triggerID" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5528",
       "triggerID" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6056",
       "triggerID" : "1041367799",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6056",
       "triggerID" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1041367799",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6189",
       "triggerID" : "1047380901",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6189",
       "triggerID" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1047380901",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 48b162c6c411b9cb5b605d006acc8d6e7d45cec8 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6189) 
   * 2e89b137fc1ede6cc5bd95fdf2e414eea706b091 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] hudi-bot removed a comment on pull request #4679: [HUDI-3315] RFC-35 Part-1 Support bucket index in Flink writer

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5471",
       "triggerID" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5491",
       "triggerID" : "1020748430",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5492",
       "triggerID" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5493",
       "triggerID" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5528",
       "triggerID" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6056",
       "triggerID" : "1041367799",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6056",
       "triggerID" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1041367799",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6189",
       "triggerID" : "1047380901",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6189",
       "triggerID" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1047380901",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6195",
       "triggerID" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6196",
       "triggerID" : "1047494529",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6195",
       "triggerID" : "1047494529",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6271",
       "triggerID" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6272",
       "triggerID" : "1049827248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6271",
       "triggerID" : "1049827248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6299",
       "triggerID" : "1050610982",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "8120f64681ac4e42a075d2cc04b036f737622ccd",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6388",
       "triggerID" : "8120f64681ac4e42a075d2cc04b036f737622ccd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8120f64681ac4e42a075d2cc04b036f737622ccd",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6390",
       "triggerID" : "1054218095",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "8120f64681ac4e42a075d2cc04b036f737622ccd",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6388",
       "triggerID" : "1054218095",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * 8120f64681ac4e42a075d2cc04b036f737622ccd Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6390) Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6388) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] hudi-bot commented on pull request #4679: [HUDI-3315] RFC-35 Part-1 Support bucket index in Flink writer

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5471",
       "triggerID" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5491",
       "triggerID" : "1020748430",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5492",
       "triggerID" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5493",
       "triggerID" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5528",
       "triggerID" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6056",
       "triggerID" : "1041367799",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6056",
       "triggerID" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1041367799",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6189",
       "triggerID" : "1047380901",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6189",
       "triggerID" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1047380901",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6195",
       "triggerID" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6196",
       "triggerID" : "1047494529",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6195",
       "triggerID" : "1047494529",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6271",
       "triggerID" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6272",
       "triggerID" : "1049827248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6271",
       "triggerID" : "1049827248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6299",
       "triggerID" : "1050610982",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "8120f64681ac4e42a075d2cc04b036f737622ccd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6388",
       "triggerID" : "8120f64681ac4e42a075d2cc04b036f737622ccd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8120f64681ac4e42a075d2cc04b036f737622ccd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6390",
       "triggerID" : "1054218095",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "8120f64681ac4e42a075d2cc04b036f737622ccd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6388",
       "triggerID" : "1054218095",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3340d3ac9a0d00b0f217a0dffc63b24960f39904",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6429",
       "triggerID" : "3340d3ac9a0d00b0f217a0dffc63b24960f39904",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 3340d3ac9a0d00b0f217a0dffc63b24960f39904 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6429) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] hudi-bot removed a comment on pull request #4679: [HUDI-3315] RFC-35 Part-1 Support bucket index in Flink writer

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5471",
       "triggerID" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5491",
       "triggerID" : "1020748430",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5492",
       "triggerID" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5493",
       "triggerID" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5528",
       "triggerID" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6056",
       "triggerID" : "1041367799",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6056",
       "triggerID" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1041367799",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6189",
       "triggerID" : "1047380901",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6189",
       "triggerID" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1047380901",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6195",
       "triggerID" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6196",
       "triggerID" : "1047494529",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6195",
       "triggerID" : "1047494529",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6271",
       "triggerID" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6272",
       "triggerID" : "1049827248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6271",
       "triggerID" : "1049827248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6299",
       "triggerID" : "1050610982",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "8120f64681ac4e42a075d2cc04b036f737622ccd",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6388",
       "triggerID" : "8120f64681ac4e42a075d2cc04b036f737622ccd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8120f64681ac4e42a075d2cc04b036f737622ccd",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6390",
       "triggerID" : "1054218095",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "8120f64681ac4e42a075d2cc04b036f737622ccd",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6388",
       "triggerID" : "1054218095",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3340d3ac9a0d00b0f217a0dffc63b24960f39904",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "3340d3ac9a0d00b0f217a0dffc63b24960f39904",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 8120f64681ac4e42a075d2cc04b036f737622ccd Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6390) Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6388) 
   * 3340d3ac9a0d00b0f217a0dffc63b24960f39904 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] hudi-bot commented on pull request #4679: [RFC-35] Make Flink writer stream friendly

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






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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] hudi-bot removed a comment on pull request #4679: [HUDI-3315] RFC-35 Make Flink writer stream friendly

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5471",
       "triggerID" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5491",
       "triggerID" : "1020748430",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5492",
       "triggerID" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5493",
       "triggerID" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 7810bd0c529dc44094e13650696d585d21b23c82 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5493) 
   * 4d49dbe71e689f0d56d3447bc5c4367db35c33af UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] hudi-bot commented on pull request #4679: [HUDI-3315] RFC-35 Make Flink writer stream friendly

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5471",
       "triggerID" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5491",
       "triggerID" : "1020748430",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5492",
       "triggerID" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 919d42914185e265b21d6124090bbda7a882f0f9 Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5492) 
   * 7810bd0c529dc44094e13650696d585d21b23c82 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] hudi-bot commented on pull request #4679: [RFC-35] Make Flink writer stream friendly

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5471",
       "triggerID" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d4f6e12cca613145904db85abc3e1bd68a08e823 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5471) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] hudi-bot removed a comment on pull request #4679: [HUDI-3315] RFC-35 Make Flink writer stream friendly

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5471",
       "triggerID" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5491",
       "triggerID" : "1020748430",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5492",
       "triggerID" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5493",
       "triggerID" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 919d42914185e265b21d6124090bbda7a882f0f9 Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5492) 
   * 7810bd0c529dc44094e13650696d585d21b23c82 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5493) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] garyli1019 commented on a change in pull request #4679: [HUDI-3315] RFC-35 Make Flink writer stream friendly

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



##########
File path: hudi-flink/src/main/java/org/apache/hudi/table/HoodieTableSink.java
##########
@@ -84,12 +85,20 @@ public SinkRuntimeProvider getSinkRuntimeProvider(Context context) {
       // default parallelism
       int parallelism = dataStream.getExecutionConfig().getParallelism();
       DataStream<Object> pipeline;
-
-      // bootstrap
-      final DataStream<HoodieRecord> hoodieRecordDataStream =
-          Pipelines.bootstrap(conf, rowType, parallelism, dataStream, context.isBounded(), overwrite);
-      // write pipeline
-      pipeline = Pipelines.hoodieStreamWrite(conf, parallelism, hoodieRecordDataStream);
+      if (OptionsResolver.isBucketIndexTable(conf)) {
+        if (!OptionsResolver.isMorTable(conf)) {
+          throw new HoodieNotSupportedException("Bucket index only support MOR table type.");

Review comment:
       sure




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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] hudi-bot removed a comment on pull request #4679: [HUDI-2450] RFC-35 Make Flink writer stream friendly

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5471",
       "triggerID" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5491",
       "triggerID" : "1020748430",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d4f6e12cca613145904db85abc3e1bd68a08e823 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5471) Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5491) 
   * 919d42914185e265b21d6124090bbda7a882f0f9 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] hudi-bot commented on pull request #4679: [HUDI-2450] RFC-35 Make Flink writer stream friendly

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5471",
       "triggerID" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5491",
       "triggerID" : "1020748430",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d4f6e12cca613145904db85abc3e1bd68a08e823 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5471) Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5491) 
   * 919d42914185e265b21d6124090bbda7a882f0f9 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] hudi-bot removed a comment on pull request #4679: [RFC-35] Make Flink writer stream friendly

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5471",
       "triggerID" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d4f6e12cca613145904db85abc3e1bd68a08e823 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5471) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] hudi-bot commented on pull request #4679: [HUDI-3315] RFC-35 Make Flink writer stream friendly

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5471",
       "triggerID" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5491",
       "triggerID" : "1020748430",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5492",
       "triggerID" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d4f6e12cca613145904db85abc3e1bd68a08e823 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5471) Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5491) 
   * 919d42914185e265b21d6124090bbda7a882f0f9 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5492) 
   * 7810bd0c529dc44094e13650696d585d21b23c82 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] hudi-bot commented on pull request #4679: [HUDI-3315] RFC-35 Make Flink writer stream friendly

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5471",
       "triggerID" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5491",
       "triggerID" : "1020748430",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5492",
       "triggerID" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5493",
       "triggerID" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 919d42914185e265b21d6124090bbda7a882f0f9 Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5492) 
   * 7810bd0c529dc44094e13650696d585d21b23c82 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5493) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] hudi-bot commented on pull request #4679: [RFC-35] Make Flink writer stream friendly

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5471",
       "triggerID" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5491",
       "triggerID" : "1020748430",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * d4f6e12cca613145904db85abc3e1bd68a08e823 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5471) Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5491) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] hudi-bot removed a comment on pull request #4679: [HUDI-3315] RFC-35 Make Flink writer stream friendly

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5471",
       "triggerID" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5491",
       "triggerID" : "1020748430",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5492",
       "triggerID" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d4f6e12cca613145904db85abc3e1bd68a08e823 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5471) Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5491) 
   * 919d42914185e265b21d6124090bbda7a882f0f9 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5492) 
   * 7810bd0c529dc44094e13650696d585d21b23c82 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] garyli1019 commented on a change in pull request #4679: [HUDI-3315] RFC-35 Make Flink writer stream friendly

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



##########
File path: hudi-flink/src/main/java/org/apache/hudi/configuration/FlinkOptions.java
##########
@@ -310,6 +318,20 @@ private FlinkOptions() {
           + "Actual value will be obtained by invoking .toString() on the field value. Nested fields can be specified using "
           + "the dot notation eg: `a.b.c`");
 
+  public static final ConfigOption<String> INDEX_KEY_FIELD = ConfigOptions
+      .key(HoodieIndexConfig.BUCKET_INDEX_HASH_FIELD.key())
+      .stringType()
+      .defaultValue("uuid")
+      .withDescription("Record key field. Value to be used as the `recordKey` component of `HoodieKey`.\n"
+        + "Actual value will be obtained by invoking .toString() on the field value. Nested fields can be specified using "
+        + "the dot notation eg: `a.b.c`");
+
+  public static final ConfigOption<Integer> BUCKET_INDEX_NUM_BUCKETS = ConfigOptions
+      .key(HoodieIndexConfig.BUCKET_INDEX_NUM_BUCKETS.key())
+      .intType()
+      .defaultValue(256) // default 256 buckets per partition
+      .withDescription("Hudi bucket number per partition. Only affected if using Hudi bucket index.");

Review comment:
       sure, will change it into a smaller number




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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] garyli1019 commented on a change in pull request #4679: [HUDI-3315] RFC-35 Make Flink writer stream friendly

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



##########
File path: hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/HoodieFlinkWriteClient.java
##########
@@ -96,6 +101,34 @@
   public HoodieFlinkWriteClient(HoodieEngineContext context, HoodieWriteConfig writeConfig) {
     super(context, writeConfig);
     this.bucketToHandles = new HashMap<>();
+    this.bucketToAppendHandles = new HashMap<>();
+  }
+
+  public void init(HoodieFlinkTable<T> table) {
+    table.validateUpsertSchema();
+    setOperationType(WriteOperationType.UPSERT);
+  }
+
+  public Boolean buffer(HoodieRecord<T> record, String instantTime, HoodieFlinkTable<T> table) {
+    final HoodieRecordLocation loc = record.getCurrentLocation();
+    final String fileID = loc.getFileId();
+    final String partitionPath = record.getPartitionPath();

Review comment:
       IMO we need to move the buffering logic into hoodie common client, then other engine could reuse it(a streaming API later). Kafka connect sink is also looking for a streaming way to write. Is there any advantage of putting the buffering logic into the write function that I am not aware of?




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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] hudi-bot commented on pull request #4679: [HUDI-3315] RFC-35 Make Flink writer stream friendly

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5471",
       "triggerID" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5491",
       "triggerID" : "1020748430",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5492",
       "triggerID" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5493",
       "triggerID" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5528",
       "triggerID" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 7810bd0c529dc44094e13650696d585d21b23c82 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5493) 
   * 4d49dbe71e689f0d56d3447bc5c4367db35c33af Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5528) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] hudi-bot commented on pull request #4679: [HUDI-3315] RFC-35 Make Flink writer stream friendly

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5471",
       "triggerID" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5491",
       "triggerID" : "1020748430",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5492",
       "triggerID" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5493",
       "triggerID" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5528",
       "triggerID" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6056",
       "triggerID" : "1041367799",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6056",
       "triggerID" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1041367799",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * ddbf502d11717d800971f736fa81054afa1873b7 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6056) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] hudi-bot removed a comment on pull request #4679: [HUDI-3315] RFC-35 Make Flink writer stream friendly

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5471",
       "triggerID" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5491",
       "triggerID" : "1020748430",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5492",
       "triggerID" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5493",
       "triggerID" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5528",
       "triggerID" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "",
       "status" : "CANCELED",
       "url" : "TBD",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   *  Unknown: [CANCELED](TBD) 
   * f63a0753eb2f8f25cd1cd68bf5e6356a73438588 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] hudi-bot removed a comment on pull request #4679: [HUDI-3315] RFC-35 Make Flink writer stream friendly

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5471",
       "triggerID" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5491",
       "triggerID" : "1020748430",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5492",
       "triggerID" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5493",
       "triggerID" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5528",
       "triggerID" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 4d49dbe71e689f0d56d3447bc5c4367db35c33af Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5528) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] garyli1019 commented on pull request #4679: [HUDI-3315] RFC-35 Make Flink writer stream friendly

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


   @hudi-bot run azure


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] hudi-bot commented on pull request #4679: [HUDI-3315] RFC-35 Make Flink writer stream friendly

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5471",
       "triggerID" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5491",
       "triggerID" : "1020748430",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5492",
       "triggerID" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5493",
       "triggerID" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5528",
       "triggerID" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6056",
       "triggerID" : "1041367799",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6056",
       "triggerID" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1041367799",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "",
       "status" : "CANCELED",
       "url" : "TBD",
       "triggerID" : "1047380901",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6189",
       "triggerID" : "1047380901",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6189",
       "triggerID" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   *  Unknown: [CANCELED](TBD) 
   * 48b162c6c411b9cb5b605d006acc8d6e7d45cec8 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6189) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] hudi-bot removed a comment on pull request #4679: [HUDI-3315] RFC-35 Make Flink writer stream friendly

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5471",
       "triggerID" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5491",
       "triggerID" : "1020748430",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5492",
       "triggerID" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5493",
       "triggerID" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5528",
       "triggerID" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6056",
       "triggerID" : "1041367799",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6056",
       "triggerID" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1041367799",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * ddbf502d11717d800971f736fa81054afa1873b7 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6056) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] hudi-bot removed a comment on pull request #4679: [HUDI-3315] RFC-35 Make Flink writer stream friendly

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5471",
       "triggerID" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5491",
       "triggerID" : "1020748430",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5492",
       "triggerID" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5493",
       "triggerID" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5528",
       "triggerID" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6056",
       "triggerID" : "1041367799",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6056",
       "triggerID" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1041367799",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6189",
       "triggerID" : "1047380901",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6189",
       "triggerID" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1047380901",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * 48b162c6c411b9cb5b605d006acc8d6e7d45cec8 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6189) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] hudi-bot commented on pull request #4679: [HUDI-3315] RFC-35 Make Flink writer stream friendly

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5471",
       "triggerID" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5491",
       "triggerID" : "1020748430",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5492",
       "triggerID" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5493",
       "triggerID" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5528",
       "triggerID" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6056",
       "triggerID" : "1041367799",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6056",
       "triggerID" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1041367799",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6189",
       "triggerID" : "1047380901",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6189",
       "triggerID" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1047380901",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6195",
       "triggerID" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6196",
       "triggerID" : "1047494529",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6195",
       "triggerID" : "1047494529",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * 2e89b137fc1ede6cc5bd95fdf2e414eea706b091 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6196) Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6195) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] hudi-bot removed a comment on pull request #4679: [HUDI-3315] RFC-35 Part-1 Support bucket index in Flink writer

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5471",
       "triggerID" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5491",
       "triggerID" : "1020748430",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5492",
       "triggerID" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5493",
       "triggerID" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5528",
       "triggerID" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6056",
       "triggerID" : "1041367799",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6056",
       "triggerID" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1041367799",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6189",
       "triggerID" : "1047380901",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6189",
       "triggerID" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1047380901",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6195",
       "triggerID" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6196",
       "triggerID" : "1047494529",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6195",
       "triggerID" : "1047494529",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * 2e89b137fc1ede6cc5bd95fdf2e414eea706b091 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6196) Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6195) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] allenxyang commented on pull request #4679: [HUDI-3315] RFC-35 Part-1 Support bucket index in Flink writer

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


   This is an automatic reply, confirming that your e-mail was received, I will get back to you ASAP.Thank you ! Allen


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] allenxyang commented on a change in pull request #4679: [HUDI-3315] RFC-35 Part-1 Support bucket index in Flink writer

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



##########
File path: hudi-flink/src/main/java/org/apache/hudi/sink/utils/Pipelines.java
##########
@@ -189,21 +191,32 @@
   }
 
   public static DataStream<Object> hoodieStreamWrite(Configuration conf, int defaultParallelism, DataStream<HoodieRecord> dataStream) {
-    WriteOperatorFactory<HoodieRecord> operatorFactory = StreamWriteOperator.getFactory(conf);
-    return dataStream
+    if (OptionsResolver.isBucketIndexType(conf)) {

Review comment:
       After I used this patch, reported this error. I don't know why.
   
   Caused by: java.lang.RuntimeException: The timer service has not been initialized.
   	at org.apache.flink.streaming.api.operators.AbstractStreamOperator.getInternalTimerService(AbstractStreamOperator.java:616) 
   	at org.apache.flink.streaming.api.operators.KeyedProcessOperator.open(KeyedProcessOperator.java:62)
   	at org.apache.flink.streaming.runtime.tasks.OperatorChain.initializeStateAndOpenOperators(OperatorChain.java:428)
   	at org.apache.flink.streaming.runtime.tasks.StreamTask.lambda$beforeInvoke$2(StreamTask.java:555) 
   	at org.apache.flink.streaming.runtime.tasks.StreamTaskActionExecutor$1.runThrowing(StreamTaskActionExecutor.java:50) 
   	at org.apache.flink.streaming.runtime.tasks.StreamTask.beforeInvoke(StreamTask.java:545)
   	at org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:585) 
   	at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:765)
   	at org.apache.flink.runtime.taskmanager.Task.run(Task.java:580)




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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] danny0405 commented on a change in pull request #4679: [HUDI-3315] RFC-35 Part-1 Support bucket index in Flink writer

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



##########
File path: hudi-flink/src/main/java/org/apache/hudi/sink/utils/Pipelines.java
##########
@@ -189,21 +191,32 @@
   }
 
   public static DataStream<Object> hoodieStreamWrite(Configuration conf, int defaultParallelism, DataStream<HoodieRecord> dataStream) {
-    WriteOperatorFactory<HoodieRecord> operatorFactory = StreamWriteOperator.getFactory(conf);
-    return dataStream

Review comment:
       Can we have a separate method for hash index and not modify these two methods ? The methods are already too complex i think.




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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] garyli1019 commented on a change in pull request #4679: [HUDI-3315] RFC-35 Part-1 Support bucket index in Flink writer

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



##########
File path: hudi-flink/src/main/java/org/apache/hudi/sink/utils/Pipelines.java
##########
@@ -189,21 +191,32 @@
   }
 
   public static DataStream<Object> hoodieStreamWrite(Configuration conf, int defaultParallelism, DataStream<HoodieRecord> dataStream) {
-    WriteOperatorFactory<HoodieRecord> operatorFactory = StreamWriteOperator.getFactory(conf);
-    return dataStream

Review comment:
       man you forgot your previous comment lol. It used to be separated and you suggested to put them into `hoodieStreamWrite`. Let me know if I misunderstood. 




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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] garyli1019 commented on a change in pull request #4679: [HUDI-3315] RFC-35 Part-1 Support bucket index in Flink writer

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



##########
File path: hudi-flink/src/main/java/org/apache/hudi/configuration/FlinkOptions.java
##########
@@ -310,6 +318,20 @@ private FlinkOptions() {
           + "Actual value will be obtained by invoking .toString() on the field value. Nested fields can be specified using "
           + "the dot notation eg: `a.b.c`");
 
+  public static final ConfigOption<String> INDEX_KEY_FIELD = ConfigOptions
+      .key(HoodieIndexConfig.BUCKET_INDEX_HASH_FIELD.key())
+      .stringType()
+      .defaultValue("uuid")

Review comment:
       ok, two vote to define this. fixed




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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] hudi-bot removed a comment on pull request #4679: [HUDI-3315] RFC-35 Part-1 Support bucket index in Flink writer

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5471",
       "triggerID" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d4f6e12cca613145904db85abc3e1bd68a08e823",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5491",
       "triggerID" : "1020748430",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5492",
       "triggerID" : "919d42914185e265b21d6124090bbda7a882f0f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5493",
       "triggerID" : "7810bd0c529dc44094e13650696d585d21b23c82",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5528",
       "triggerID" : "4d49dbe71e689f0d56d3447bc5c4367db35c33af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6030",
       "triggerID" : "f63a0753eb2f8f25cd1cd68bf5e6356a73438588",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1040300417",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6056",
       "triggerID" : "1041367799",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6056",
       "triggerID" : "ddbf502d11717d800971f736fa81054afa1873b7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1041367799",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6189",
       "triggerID" : "1047380901",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6189",
       "triggerID" : "48b162c6c411b9cb5b605d006acc8d6e7d45cec8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "1047380901",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6195",
       "triggerID" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6196",
       "triggerID" : "1047494529",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "2e89b137fc1ede6cc5bd95fdf2e414eea706b091",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6195",
       "triggerID" : "1047494529",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6271",
       "triggerID" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6272",
       "triggerID" : "1049827248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6271",
       "triggerID" : "1049827248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "96cbbb8e39d47a513d2ad5c0a43ffa06d249f1f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6299",
       "triggerID" : "1050610982",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "8120f64681ac4e42a075d2cc04b036f737622ccd",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6388",
       "triggerID" : "8120f64681ac4e42a075d2cc04b036f737622ccd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8120f64681ac4e42a075d2cc04b036f737622ccd",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6390",
       "triggerID" : "1054218095",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "8120f64681ac4e42a075d2cc04b036f737622ccd",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6388",
       "triggerID" : "1054218095",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3340d3ac9a0d00b0f217a0dffc63b24960f39904",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6429",
       "triggerID" : "3340d3ac9a0d00b0f217a0dffc63b24960f39904",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 8120f64681ac4e42a075d2cc04b036f737622ccd Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6390) Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6388) 
   * 3340d3ac9a0d00b0f217a0dffc63b24960f39904 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6429) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] garyli1019 commented on pull request #4679: [HUDI-3315] RFC-35 Part-1 Support bucket index in Flink writer

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


   @wxplovecc hi, this issue looks like not related to this PR, would you submit an issue then we can take a look.


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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