[jira] [Commented] (HUDI-3461) The archived timeline for flink streaming reader should not be reused

2022-02-21 Thread Danny Chen (Jira)


[ 
https://issues.apache.org/jira/browse/HUDI-3461?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17495935#comment-17495935
 ] 

Danny Chen commented on HUDI-3461:
--

Fixed via master branch: 4affdd0c8f02ccca4705515dcb6492c199e2cede

> The archived timeline for flink streaming reader should not be reused
> -
>
> Key: HUDI-3461
> URL: https://issues.apache.org/jira/browse/HUDI-3461
> Project: Apache Hudi
>  Issue Type: Bug
>  Components: flink
>Reporter: Danny Chen
>Priority: Major
>  Labels: pull-request-available
> Fix For: 0.11.0
>
>
> see the error stack trace:
> {code:xml}
> 2022-02-21 14:21:30
> java.util.NoSuchElementException: No value present in Option
>   at org.apache.hudi.common.util.Option.get(Option.java:88)
>   at 
> org.apache.hudi.hadoop.utils.HoodieInputFormatUtils.getCommitMetadata(HoodieInputFormatUtils.java:572)
>   at 
> org.apache.hudi.sink.partitioner.profile.WriteProfiles.getCommitMetadata(WriteProfiles.java:192)
>   at 
> org.apache.hudi.source.IncrementalInputSplits.lambda$getArchivedMetadata$5(IncrementalInputSplits.java:270)
>   at 
> java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:193)
>   at 
> java.util.stream.ReferencePipeline$2$1.accept(ReferencePipeline.java:175)
>   at 
> java.util.ArrayList$ArrayListSpliterator.forEachRemaining(ArrayList.java:1384)
>   at java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:482)
>   at 
> java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:472)
>   at 
> java.util.stream.ReduceOps$ReduceOp.evaluateSequential(ReduceOps.java:708)
>   at java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234)
>   at 
> java.util.stream.ReferencePipeline.collect(ReferencePipeline.java:566)
>   at 
> org.apache.hudi.source.IncrementalInputSplits.getArchivedMetadata(IncrementalInputSplits.java:270)
>   at 
> org.apache.hudi.source.IncrementalInputSplits.inputSplits(IncrementalInputSplits.java:184)
>   at 
> org.apache.hudi.source.StreamReadMonitoringFunction.monitorDirAndForwardSplits(StreamReadMonitoringFunction.java:195)
>   at 
> org.apache.hudi.source.StreamReadMonitoringFunction.run(StreamReadMonitoringFunction.java:168)
>   at 
> org.apache.flink.streaming.api.operators.StreamSource.run(StreamSource.java:110)
>   at 
> org.apache.flink.streaming.api.operators.StreamSource.run(StreamSource.java:66)
>   at 
> org.apache.flink.streaming.runtime.tasks.SourceStreamTask$LegacySourceFunctionThread.run(SourceStreamTask.java:269)
> {code}



--
This message was sent by Atlassian Jira
(v8.20.1#820001)


[jira] [Resolved] (HUDI-3461) The archived timeline for flink streaming reader should not be reused

2022-02-21 Thread Danny Chen (Jira)


 [ 
https://issues.apache.org/jira/browse/HUDI-3461?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Danny Chen resolved HUDI-3461.
--

> The archived timeline for flink streaming reader should not be reused
> -
>
> Key: HUDI-3461
> URL: https://issues.apache.org/jira/browse/HUDI-3461
> Project: Apache Hudi
>  Issue Type: Bug
>  Components: flink
>Reporter: Danny Chen
>Priority: Major
>  Labels: pull-request-available
> Fix For: 0.11.0
>
>
> see the error stack trace:
> {code:xml}
> 2022-02-21 14:21:30
> java.util.NoSuchElementException: No value present in Option
>   at org.apache.hudi.common.util.Option.get(Option.java:88)
>   at 
> org.apache.hudi.hadoop.utils.HoodieInputFormatUtils.getCommitMetadata(HoodieInputFormatUtils.java:572)
>   at 
> org.apache.hudi.sink.partitioner.profile.WriteProfiles.getCommitMetadata(WriteProfiles.java:192)
>   at 
> org.apache.hudi.source.IncrementalInputSplits.lambda$getArchivedMetadata$5(IncrementalInputSplits.java:270)
>   at 
> java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:193)
>   at 
> java.util.stream.ReferencePipeline$2$1.accept(ReferencePipeline.java:175)
>   at 
> java.util.ArrayList$ArrayListSpliterator.forEachRemaining(ArrayList.java:1384)
>   at java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:482)
>   at 
> java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:472)
>   at 
> java.util.stream.ReduceOps$ReduceOp.evaluateSequential(ReduceOps.java:708)
>   at java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234)
>   at 
> java.util.stream.ReferencePipeline.collect(ReferencePipeline.java:566)
>   at 
> org.apache.hudi.source.IncrementalInputSplits.getArchivedMetadata(IncrementalInputSplits.java:270)
>   at 
> org.apache.hudi.source.IncrementalInputSplits.inputSplits(IncrementalInputSplits.java:184)
>   at 
> org.apache.hudi.source.StreamReadMonitoringFunction.monitorDirAndForwardSplits(StreamReadMonitoringFunction.java:195)
>   at 
> org.apache.hudi.source.StreamReadMonitoringFunction.run(StreamReadMonitoringFunction.java:168)
>   at 
> org.apache.flink.streaming.api.operators.StreamSource.run(StreamSource.java:110)
>   at 
> org.apache.flink.streaming.api.operators.StreamSource.run(StreamSource.java:66)
>   at 
> org.apache.flink.streaming.runtime.tasks.SourceStreamTask$LegacySourceFunctionThread.run(SourceStreamTask.java:269)
> {code}



--
This message was sent by Atlassian Jira
(v8.20.1#820001)


[hudi] branch master updated: [HUDI-3461] The archived timeline for flink streaming reader should not be reused (#4861)

2022-02-21 Thread danny0405
This is an automated email from the ASF dual-hosted git repository.

danny0405 pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hudi.git


The following commit(s) were added to refs/heads/master by this push:
 new 4affdd0  [HUDI-3461] The archived timeline for flink streaming reader 
should not be reused (#4861)
4affdd0 is described below

commit 4affdd0c8f02ccca4705515dcb6492c199e2cede
Author: Danny Chan 
AuthorDate: Tue Feb 22 15:54:29 2022 +0800

[HUDI-3461] The archived timeline for flink streaming reader should not be 
reused (#4861)

* Before the patch, the flink streaming reader caches the meta client thus 
the archived timeline,
  when fetching the instant details from the reused timeline, the exception 
throws
* Add a method in HoodieTableMetaClient to return a fresh new archived 
timeline each time
---
 .../hudi/common/table/HoodieTableMetaClient.java   | 18 ++--
 .../table/timeline/HoodieArchivedTimeline.java | 34 +++---
 .../apache/hudi/source/IncrementalInputSplits.java | 16 ++
 3 files changed, 49 insertions(+), 19 deletions(-)

diff --git 
a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java
 
b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java
index 740d569..4c1eac7 100644
--- 
a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java
+++ 
b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java
@@ -156,7 +156,7 @@ public class HoodieTableMetaClient implements Serializable {
*/
   private void readObject(java.io.ObjectInputStream in) throws IOException, 
ClassNotFoundException {
 in.defaultReadObject();
-fs = null; // will be lazily inited
+fs = null; // will be lazily initialized
   }
 
   private void writeObject(java.io.ObjectOutputStream out) throws IOException {
@@ -330,7 +330,7 @@ public class HoodieTableMetaClient implements Serializable {
* Get the archived commits as a timeline. This is costly operation, as all 
data from the archived files are read.
* This should not be used, unless for historical debugging purposes.
*
-   * @return Active commit timeline
+   * @return Archived commit timeline
*/
   public synchronized HoodieArchivedTimeline getArchivedTimeline() {
 if (archivedTimeline == null) {
@@ -340,6 +340,20 @@ public class HoodieTableMetaClient implements Serializable 
{
   }
 
   /**
+   * Returns fresh new archived commits as a timeline from startTs (inclusive).
+   *
+   * This is costly operation if really early endTs is specified.
+   * Be caution to use this only when the time range is short.
+   *
+   * This method is not thread safe.
+   *
+   * @return Archived commit timeline
+   */
+  public HoodieArchivedTimeline getArchivedTimeline(String startTs) {
+return new HoodieArchivedTimeline(this, startTs);
+  }
+
+  /**
* Validate table properties.
* @param properties Properties from writeConfig.
* @param operationType operation type to be executed.
diff --git 
a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieArchivedTimeline.java
 
b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieArchivedTimeline.java
index 5ad3fa7..29f1665 100644
--- 
a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieArchivedTimeline.java
+++ 
b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieArchivedTimeline.java
@@ -79,13 +79,13 @@ public class HoodieArchivedTimeline extends 
HoodieDefaultTimeline {
   private static final String ACTION_TYPE_KEY = "actionType";
   private static final String ACTION_STATE = "actionState";
   private HoodieTableMetaClient metaClient;
-  private Map readCommits = new HashMap<>();
+  private final Map readCommits = new HashMap<>();
 
   private static final Logger LOG = 
LogManager.getLogger(HoodieArchivedTimeline.class);
 
   /**
-   * Loads instants between (startTs, endTs].
-   * Note that there is no lazy loading, so this may not work if really long 
time range (endTs-startTs) is specified.
+   * Loads all the archived instants.
+   * Note that there is no lazy loading, so this may not work if the archived 
timeline range is really long.
* TBD: Should we enforce maximum time range?
*/
   public HoodieArchivedTimeline(HoodieTableMetaClient metaClient) {
@@ -97,6 +97,19 @@ public class HoodieArchivedTimeline extends 
HoodieDefaultTimeline {
   }
 
   /**
+   * Loads completed instants from startTs(inclusive).
+   * Note that there is no lazy loading, so this may not work if really early 
startTs is specified.
+   */
+  public HoodieArchivedTimeline(HoodieTableMetaClient metaClient, String 
startTs) {
+this.metaClient = metaClient;
+setInstants(loadInstants(new StartTsFilter(startTs), true,
+record -> 
HoodieInstant.State.COMPLETED.toString().equals(record.get(ACTION_STATE).toString(;
+// 

[GitHub] [hudi] danny0405 merged pull request #4861: [HUDI-3461] The archived timeline for flink streaming reader should n…

2022-02-21 Thread GitBox


danny0405 merged pull request #4861:
URL: https://github.com/apache/hudi/pull/4861


   


-- 
This is an automated message from the 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-unsubscr...@hudi.apache.org

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




[GitHub] [hudi] glory9211 commented on issue #4839: Hudi upsert doesnt trigger compaction for MOR

2022-02-21 Thread GitBox


glory9211 commented on issue #4839:
URL: https://github.com/apache/hudi/issues/4839#issuecomment-1047513904


   What should be the approach in case you have multiple sinks, i.e you split 
your incoming data into multiple hudi tables. How would you store these without 
using foreachBatch ?


-- 
This is an automated message from the 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-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@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

2022-02-21 Thread GitBox


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 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 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 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 extends StreamWriteFunction {
+
+  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 bucketToFileIDMap;
+
+  /**
+   * Constructs a BucketStreamWriteFunction.
+   *
+   * @param config The config options
+   */
+  public BucketStreamWriteFunction(Configuration config) {
+super(config);
+this.bucketToFileIDMap = new HashMap();
+  }
+
+  @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

[GitHub] [hudi] wjcwin commented on issue #4853: [SUPPORT] documentation about the Flink Hudi pipeline util

2022-02-21 Thread GitBox


wjcwin commented on issue #4853:
URL: https://github.com/apache/hudi/issues/4853#issuecomment-1047511470


   > > > Does add java documents solve your problem ?
   > > 
   > > 
   > > I don't know how to use these methods to build a standard task,If there 
are some guidelines, it will be of great help to me
   > 
   > I see, i have created a JIRA issue: 
https://issues.apache.org/jira/browse/HUDI-3474 to add more documents/usages to 
this tool class for building a complete pipeline ~
   
   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-unsubscr...@hudi.apache.org

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




[GitHub] [hudi] danny0405 edited a comment on issue #4853: [SUPPORT] documentation about the Flink Hudi pipeline util

2022-02-21 Thread GitBox


danny0405 edited a comment on issue #4853:
URL: https://github.com/apache/hudi/issues/4853#issuecomment-1047505698


   > > Does add java documents solve your problem ?
   > 
   > I don't know how to use these methods to build a standard task,If there 
are some guidelines, it will be of great help to me
   
   I see, i have created a JIRA issue: 
https://issues.apache.org/jira/browse/HUDI-3474
   to add more documents/usages to this tool class for building a complete 
pipeline ~


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

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

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




[jira] [Created] (HUDI-3474) Add more document to Pipelines for the usage of this tool to build a write pipeline

2022-02-21 Thread Danny Chen (Jira)
Danny Chen created HUDI-3474:


 Summary: Add more document to Pipelines for the usage of this tool 
to build a write pipeline
 Key: HUDI-3474
 URL: https://issues.apache.org/jira/browse/HUDI-3474
 Project: Apache Hudi
  Issue Type: Improvement
  Components: docs, flink
Reporter: Danny Chen
 Fix For: 0.11.0






--
This message was sent by Atlassian Jira
(v8.20.1#820001)


[GitHub] [hudi] danny0405 commented on issue #4853: [SUPPORT] documentation about the Flink Hudi pipeline util

2022-02-21 Thread GitBox


danny0405 commented on issue #4853:
URL: https://github.com/apache/hudi/issues/4853#issuecomment-1047505698


   > > Does add java documents solve your problem ?
   > 
   > I don't know how to use these methods to build a standard task,If there 
are some guidelines, it will be of great help to me
   
   I see, i have created a JIRA issue to add more documents/usages to this tool 
class for building a complete pipeline ~


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

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

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




[GitHub] [hudi] danny0405 edited a comment on issue #4863: [SUPPORT] Compaction and rollback with Flink cause data loss

2022-02-21 Thread GitBox


danny0405 edited a comment on issue #4863:
URL: https://github.com/apache/hudi/issues/4863#issuecomment-1047504055


   Did you mean the coordinator rollback the compaction instant ? This is not 
as expected because we have excluded the compaction and clustering instant, 
see:https://github.com/apache/hudi/blob/4d1f74ebeaee857380f69d7c596eaaf0135ca59e/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java#L973
 and 
https://github.com/apache/hudi/blob/4d1f74ebeaee857380f69d7c596eaaf0135ca59e/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java#L1028
 the compaction rollback actually happens in `CompactionPlanOperator`: 
https://github.com/apache/hudi/blob/4d1f74ebeaee857380f69d7c596eaaf0135ca59e/hudi-flink/src/main/java/org/apache/hudi/sink/compact/CompactionPlanOperator.java#L73
   
   But i believe there was something wrong here, let's dig deeper into 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-unsubscr...@hudi.apache.org

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




[GitHub] [hudi] danny0405 commented on issue #4863: [SUPPORT] Compaction and rollback with Flink cause data loss

2022-02-21 Thread GitBox


danny0405 commented on issue #4863:
URL: https://github.com/apache/hudi/issues/4863#issuecomment-1047504055


   Did you mean the coordinator rollback the compaction instant ? This is not 
as expected because we have excluded the compaction and clustering instant, 
see:https://github.com/apache/hudi/blob/4d1f74ebeaee857380f69d7c596eaaf0135ca59e/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java#L973
 and 
https://github.com/apache/hudi/blob/4d1f74ebeaee857380f69d7c596eaaf0135ca59e/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java#L1028
 the compaction rollback actually happens in `CompactionPlanOperator`: 
https://github.com/apache/hudi/blob/4d1f74ebeaee857380f69d7c596eaaf0135ca59e/hudi-flink/src/main/java/org/apache/hudi/sink/compact/CompactionPlanOperator.java#L73


-- 
This is an automated message from the 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-unsubscr...@hudi.apache.org

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




[GitHub] [hudi] hudi-bot removed a comment on pull request #4861: [HUDI-3461] The archived timeline for flink streaming reader should n…

2022-02-21 Thread GitBox


hudi-bot removed a comment on pull request #4861:
URL: https://github.com/apache/hudi/pull/4861#issuecomment-1047462954


   
   ## CI report:
   
   *  Unknown: [CANCELED](TBD) 
   * 26828a92c5dd961e75bb943ca057ff0e21feabd6 Azure: 
[PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6193)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
This is an automated message from the 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-unsubscr...@hudi.apache.org

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




[GitHub] [hudi] hudi-bot commented on pull request #4861: [HUDI-3461] The archived timeline for flink streaming reader should n…

2022-02-21 Thread GitBox


hudi-bot commented on pull request #4861:
URL: https://github.com/apache/hudi/pull/4861#issuecomment-1047501220


   
   ## CI report:
   
   * 26828a92c5dd961e75bb943ca057ff0e21feabd6 Azure: 
[SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6193)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
This is an automated message from the 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-unsubscr...@hudi.apache.org

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




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

2022-02-21 Thread GitBox


hudi-bot commented on pull request #4679:
URL: https://github.com/apache/hudi/pull/4679#issuecomment-1047498557


   
   ## 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)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
This is an automated message from the 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-unsubscr...@hudi.apache.org

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




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

2022-02-21 Thread GitBox


hudi-bot removed a comment on pull request #4679:
URL: https://github.com/apache/hudi/pull/4679#issuecomment-1047496264


   
   ## 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)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
This is an automated message from the 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-unsubscr...@hudi.apache.org

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




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

2022-02-21 Thread GitBox


hudi-bot removed a comment on pull request #4679:
URL: https://github.com/apache/hudi/pull/4679#issuecomment-1047494080


   
   ## CI report:
   
   * 48b162c6c411b9cb5b605d006acc8d6e7d45cec8 Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6189)
 
   * 2e89b137fc1ede6cc5bd95fdf2e414eea706b091 UNKNOWN
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
This is an automated message from the 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-unsubscr...@hudi.apache.org

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




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

2022-02-21 Thread GitBox


hudi-bot commented on pull request #4679:
URL: https://github.com/apache/hudi/pull/4679#issuecomment-1047496264


   
   ## 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)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
This is an automated message from the 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-unsubscr...@hudi.apache.org

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




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

2022-02-21 Thread GitBox


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-unsubscr...@hudi.apache.org

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




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

2022-02-21 Thread GitBox


hudi-bot removed a comment on pull request #4679:
URL: https://github.com/apache/hudi/pull/4679#issuecomment-1047421762


   
   ## CI report:
   
   * 48b162c6c411b9cb5b605d006acc8d6e7d45cec8 Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6189)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
This is an automated message from the 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-unsubscr...@hudi.apache.org

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




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

2022-02-21 Thread GitBox


hudi-bot commented on pull request #4679:
URL: https://github.com/apache/hudi/pull/4679#issuecomment-1047494080


   
   ## CI report:
   
   * 48b162c6c411b9cb5b605d006acc8d6e7d45cec8 Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6189)
 
   * 2e89b137fc1ede6cc5bd95fdf2e414eea706b091 UNKNOWN
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
This is an automated message from the 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-unsubscr...@hudi.apache.org

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




[GitHub] [hudi] wjcwin commented on issue #4853: [SUPPORT] documentation about the Flink Hudi pipeline util

2022-02-21 Thread GitBox


wjcwin commented on issue #4853:
URL: https://github.com/apache/hudi/issues/4853#issuecomment-1047485223


   > Does add java documents solve your problem ?
   
   I don't know how to use these methods to build a standard task,If there are 
some guidelines, it will be of great help to me


-- 
This is an automated message from the 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-unsubscr...@hudi.apache.org

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




[GitHub] [hudi] danny0405 commented on issue #4855: [SUPPORT] mor table parquet file max file size does not go into effect

2022-02-21 Thread GitBox


danny0405 commented on issue #4855:
URL: https://github.com/apache/hudi/issues/4855#issuecomment-1047484493


   Yes, the file size control is not accurate, how much size does the actual 
file size exceed your desired threshold ?


-- 
This is an automated message from the 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-unsubscr...@hudi.apache.org

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




[GitHub] [hudi] danny0405 commented on issue #4853: [SUPPORT] documentation about the Flink Hudi pipeline util

2022-02-21 Thread GitBox


danny0405 commented on issue #4853:
URL: https://github.com/apache/hudi/issues/4853#issuecomment-1047483062


   Does add java documents solve your problem ?


-- 
This is an automated message from the 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-unsubscr...@hudi.apache.org

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




[GitHub] [hudi] danny0405 commented on issue #4835: [SUPPORT] failed to read log file

2022-02-21 Thread GitBox


danny0405 commented on issue #4835:
URL: https://github.com/apache/hudi/issues/4835#issuecomment-1047482617


   I see the file you read is an empty file, have no idea yet how this file is 
generated.


-- 
This is an automated message from the 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-unsubscr...@hudi.apache.org

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




[GitHub] [hudi] wjcwin commented on issue #4853: [SUPPORT] documentation about the Flink Hudi pipeline util

2022-02-21 Thread GitBox


wjcwin commented on issue #4853:
URL: https://github.com/apache/hudi/issues/4853#issuecomment-1047478556


   > What do you mean for flink hudi pipeline, do you mean the operators ?
   
   org.apache.hudi.sink.utils.Pipelines 


-- 
This is an automated message from the 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-unsubscr...@hudi.apache.org

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




[GitHub] [hudi] danny0405 commented on issue #4816: [SUPPORT] hudi flink sql suport for _hoodie_commit_time Column

2022-02-21 Thread GitBox


danny0405 commented on issue #4816:
URL: https://github.com/apache/hudi/issues/4816#issuecomment-1047478211


   The metadata field is added for time based filtering, because for streaming 
read we need to filter the data stream for each reading batch ~


-- 
This is an automated message from the 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-unsubscr...@hudi.apache.org

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




[GitHub] [hudi] danny0405 commented on issue #4853: [SUPPORT] documentation about the Flink Hudi pipeline util

2022-02-21 Thread GitBox


danny0405 commented on issue #4853:
URL: https://github.com/apache/hudi/issues/4853#issuecomment-1047477001


   What do you mean for flink hudi pipeline, do you mean the operators ?


-- 
This is an automated message from the 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-unsubscr...@hudi.apache.org

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




[jira] [Created] (HUDI-3473) Spark reader adapter to the meta field '__hoodie_operation' and merge the intermediate changes

2022-02-21 Thread Danny Chen (Jira)
Danny Chen created HUDI-3473:


 Summary: Spark reader adapter to the meta field 
'__hoodie_operation' and merge the intermediate changes
 Key: HUDI-3473
 URL: https://issues.apache.org/jira/browse/HUDI-3473
 Project: Apache Hudi
  Issue Type: Improvement
  Components: spark
Reporter: Danny Chen
 Fix For: 0.11.0






--
This message was sent by Atlassian Jira
(v8.20.1#820001)


[GitHub] [hudi] danny0405 commented on issue #4868: [SUPPORT] After the changlog mode is turned on, the reading is repeated

2022-02-21 Thread GitBox


danny0405 commented on issue #4868:
URL: https://github.com/apache/hudi/issues/4868#issuecomment-1047475853


   I have created a JIRA to address this problem: 
https://issues.apache.org/jira/browse/HUDI-3473.


-- 
This is an automated message from the 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-unsubscr...@hudi.apache.org

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




[GitHub] [hudi] danny0405 commented on issue #4868: [SUPPORT] After the changlog mode is turned on, the reading is repeated

2022-02-21 Thread GitBox


danny0405 commented on issue #4868:
URL: https://github.com/apache/hudi/issues/4868#issuecomment-1047474072


   Yes, the spark reader should adapter to the hoodie operations.


-- 
This is an automated message from the 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-unsubscr...@hudi.apache.org

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




[GitHub] [hudi] danny0405 commented on issue #4858: [SUPPORT] data loss occurred flink hudi

2022-02-21 Thread GitBox


danny0405 commented on issue #4858:
URL: https://github.com/apache/hudi/issues/4858#issuecomment-1047473610


   Yes, COW table type does not have this problem, but the throughput is not 
high as MOR table.


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

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

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




[jira] [Commented] (HUDI-3411) Incorrect Record Key Field property Handling

2022-02-21 Thread todd (Jira)


[ 
https://issues.apache.org/jira/browse/HUDI-3411?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17495881#comment-17495881
 ] 

todd commented on HUDI-3411:


agree。in 
addition,KeyGenUtils#getRecordKey(org.apache.avro.generic.GenericRecord, 
java.lang.String, boolean) and 
KeyGenUtils#getRecordKey(org.apache.avro.generic.GenericRecord, 
java.lang.String, boolean)

There are some differences in the format of the generated key, which causes the 
recordkey and the recordkey written by flink to not match correctly.

If you want to modify, how to unify.

> Incorrect Record Key Field property Handling
> 
>
> Key: HUDI-3411
> URL: https://issues.apache.org/jira/browse/HUDI-3411
> Project: Apache Hudi
>  Issue Type: Bug
>Reporter: Alexey Kudinkin
>Priority: Blocker
> Fix For: 0.11.0
>
>
> Currently `HoodieTableConfig#getRecordKeyFieldProp` returns a single String, 
> even though it could contain a *list* of columns making up composite Primary 
> Key.
> {code:java}
> public String getRecordKeyFieldProp() {
>   return getStringOrDefault(RECORDKEY_FIELDS, 
> HoodieRecord.RECORD_KEY_METADATA_FIELD);
> } {code}
>  
> Most of the callers of this method are actually not handling this correctly, 
> assuming that the Record Key is always a single field. 
> NOTE: While concatenation of CPK seems like a very natural step here, special 
> care has to be taken, since Composite PK can NOT be concatenated as strings, 
> as this might break the uniqueness constraint. 



--
This message was sent by Atlassian Jira
(v8.20.1#820001)


[jira] [Closed] (HUDI-3464) Fix wrong exception thrown from HiveSchemaProvider

2022-02-21 Thread Xianghu Wang (Jira)


 [ 
https://issues.apache.org/jira/browse/HUDI-3464?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Xianghu Wang closed HUDI-3464.
--
Resolution: Fixed

Fixed via master : 4d1f74ebeaee857380f69d7c596eaaf0135ca59e

> Fix wrong exception thrown from HiveSchemaProvider
> --
>
> Key: HUDI-3464
> URL: https://issues.apache.org/jira/browse/HUDI-3464
> Project: Apache Hudi
>  Issue Type: Bug
>  Components: deltastreamer
>Reporter: Xianghu Wang
>Assignee: Xianghu Wang
>Priority: Major
>  Labels: pull-request-available
> Fix For: 0.11.0
>
>
> step to reproduce:
>  # set  SOURCE_SCHEMA_TABLE_PROP, SOURCE_SCHEMA_DATABASE_PROP to right config;
>  # SOURCE_SCHEMA_TABLE_PROP does not exist;
>  # start the task using HiveSchemaProvider
> Caused by: java.lang.IllegalArgumentException: Property 
> hoodie.deltastreamer.schemaprovider.target.schema.hive.table not found
> at 
> org.apache.hudi.common.config.TypedProperties.checkKey(TypedProperties.java:48)
> at 
> org.apache.hudi.common.config.TypedProperties.getString(TypedProperties.java:58)
> at 
> org.apache.hudi.utilities.schema.HiveSchemaProvider.(HiveSchemaProvider.java:83)
>  
> Acturally, the right message should be SOURCE_SCHEMA_TABLE_PROP not found, 
> but it is covered by above error message



--
This message was sent by Atlassian Jira
(v8.20.1#820001)


[hudi] branch master updated (14dbbdf -> 4d1f74e)

2022-02-21 Thread wangxianghu
This is an automated email from the ASF dual-hosted git repository.

wangxianghu pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/hudi.git.


from 14dbbdf  [HUDI-2189] Adding delete partitions support to DeltaStreamer 
(#4787)
 add 4d1f74e  [HUDI-3464] Fix wrong exception thrown from 
HiveSchemaProvider (#4865)

No new revisions were added by this update.

Summary of changes:
 ...ion.java => HoodieSchemaProviderException.java} |  6 +--
 .../hudi/utilities/schema/HiveSchemaProvider.java  | 49 --
 2 files changed, 30 insertions(+), 25 deletions(-)
 copy 
hudi-utilities/src/main/java/org/apache/hudi/utilities/exception/{HoodieDeltaStreamerException.java
 => HoodieSchemaProviderException.java} (83%)


[GitHub] [hudi] wangxianghu merged pull request #4865: [HUDI-3464] Fix wrong exception thrown from HiveSchemaProvider

2022-02-21 Thread GitBox


wangxianghu merged pull request #4865:
URL: https://github.com/apache/hudi/pull/4865


   


-- 
This is an automated message from the 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-unsubscr...@hudi.apache.org

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




[GitHub] [hudi] hudi-bot removed a comment on pull request #4861: [HUDI-3461] The archived timeline for flink streaming reader should n…

2022-02-21 Thread GitBox


hudi-bot removed a comment on pull request #4861:
URL: https://github.com/apache/hudi/pull/4861#issuecomment-1047461098


   
   ## CI report:
   
   *  Unknown: [CANCELED](TBD) 
   * 26828a92c5dd961e75bb943ca057ff0e21feabd6 UNKNOWN
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
This is an automated message from the 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-unsubscr...@hudi.apache.org

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




[GitHub] [hudi] hudi-bot commented on pull request #4861: [HUDI-3461] The archived timeline for flink streaming reader should n…

2022-02-21 Thread GitBox


hudi-bot commented on pull request #4861:
URL: https://github.com/apache/hudi/pull/4861#issuecomment-1047462954


   
   ## CI report:
   
   *  Unknown: [CANCELED](TBD) 
   * 26828a92c5dd961e75bb943ca057ff0e21feabd6 Azure: 
[PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6193)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
This is an automated message from the 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-unsubscr...@hudi.apache.org

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




[GitHub] [hudi] hudi-bot removed a comment on pull request #4861: [HUDI-3461] The archived timeline for flink streaming reader should n…

2022-02-21 Thread GitBox


hudi-bot removed a comment on pull request #4861:
URL: https://github.com/apache/hudi/pull/4861#issuecomment-1047437791


   
   ## CI report:
   
   * 75f70ba1856029f7530b75cbd314597af55db867 Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6191)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
This is an automated message from the 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-unsubscr...@hudi.apache.org

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




[GitHub] [hudi] hudi-bot commented on pull request #4861: [HUDI-3461] The archived timeline for flink streaming reader should n…

2022-02-21 Thread GitBox


hudi-bot commented on pull request #4861:
URL: https://github.com/apache/hudi/pull/4861#issuecomment-1047461098


   
   ## CI report:
   
   *  Unknown: [CANCELED](TBD) 
   * 26828a92c5dd961e75bb943ca057ff0e21feabd6 UNKNOWN
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
This is an automated message from the 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-unsubscr...@hudi.apache.org

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




[GitHub] [hudi] danny0405 commented on a change in pull request #4861: [HUDI-3461] The archived timeline for flink streaming reader should n…

2022-02-21 Thread GitBox


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



##
File path: 
hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieArchivedTimeline.java
##
@@ -117,12 +130,19 @@ public static Path getArchiveLogPath(String 
archiveFolder) {
   }
 
   public void loadInstantDetailsInMemory(String startTs, String endTs) {
-loadInstants(startTs, endTs);
+setInstants(loadInstants(startTs, endTs));
   }
 
   public void loadCompletedInstantDetailsInMemory() {
-loadInstants(null, true,
+final List instants = loadInstants(null, true,
 record -> 
HoodieInstant.State.COMPLETED.toString().equals(record.get(ACTION_STATE).toString()));
+setInstants(instants);

Review comment:
   I'm shocked that this method does nothing before, the return value is 
ignored, please help to conform this @nsivabalan :)

##
File path: 
hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieArchivedTimeline.java
##
@@ -117,12 +130,19 @@ public static Path getArchiveLogPath(String 
archiveFolder) {
   }
 
   public void loadInstantDetailsInMemory(String startTs, String endTs) {
-loadInstants(startTs, endTs);
+setInstants(loadInstants(startTs, endTs));
   }
 
   public void loadCompletedInstantDetailsInMemory() {
-loadInstants(null, true,
+final List instants = loadInstants(null, true,
 record -> 
HoodieInstant.State.COMPLETED.toString().equals(record.get(ACTION_STATE).toString()));
+setInstants(instants);

Review comment:
   Oops, my fault, ignore this comment, the `readCommits` is updated in 
method `readCommit`.




-- 
This is an automated message from the 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-unsubscr...@hudi.apache.org

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




[GitHub] [hudi] danny0405 commented on pull request #4861: [HUDI-3461] The archived timeline for flink streaming reader should n…

2022-02-21 Thread GitBox


danny0405 commented on pull request #4861:
URL: https://github.com/apache/hudi/pull/4861#issuecomment-1047459666


   @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-unsubscr...@hudi.apache.org

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




[GitHub] [hudi] codope commented on issue #4437: [QUESTION] Example for CREATE TABLE on TRINO using HUDI

2022-02-21 Thread GitBox


codope commented on issue #4437:
URL: https://github.com/apache/hudi/issues/4437#issuecomment-1047448948


   @gunjdesai First of all, apologies for the late response. I missed tracking 
this issue.
   
   In its current state, we must [sync Hudi table to Hive 
metastore](https://hudi.apache.org/docs/syncing_metastore) in order to query 
through Trino. Our integration with Trino is via the hive connector. And that 
too, only for reads i.e. we cannot create hudi tables using Trino.
   
   After syncing the Hudi table to Hive metastore and placing the 
hudi-presto-bundle in the hive plugin path, we can run snapshot queries on COW 
table and read-optimized queries on MOR table right away. You can look at an 
example in our [docker demo](https://hudi.apache.org/docs/docker_demo). That 
said, we are working on a [separate Hudi 
connector](https://github.com/trinodb/trino/pull/10228) to which we'll add the 
write support after the base PR is merged.
   
   I would love to understand your requirements and help you out with the Hive 
sync and Trino setup to query Hudi tables. You can also help us by checking out 
the PR linked above and test for your tables. Feel free to directly ping me on 
Hudi slack or join our weekly [community 
syncs](https://hudi.apache.org/community/syncs).


-- 
This is an automated message from the 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-unsubscr...@hudi.apache.org

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




[GitHub] [hudi] hudi-bot commented on pull request #4861: [HUDI-3461] The archived timeline for flink streaming reader should n…

2022-02-21 Thread GitBox


hudi-bot commented on pull request #4861:
URL: https://github.com/apache/hudi/pull/4861#issuecomment-1047437791


   
   ## CI report:
   
   * 75f70ba1856029f7530b75cbd314597af55db867 Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6191)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
This is an automated message from the 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-unsubscr...@hudi.apache.org

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




[GitHub] [hudi] hudi-bot removed a comment on pull request #4861: [HUDI-3461] The archived timeline for flink streaming reader should n…

2022-02-21 Thread GitBox


hudi-bot removed a comment on pull request #4861:
URL: https://github.com/apache/hudi/pull/4861#issuecomment-1047404397


   
   ## CI report:
   
   * 805c912d164c10de9db27434d8efc7890250f3eb Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6163)
 
   * 75f70ba1856029f7530b75cbd314597af55db867 Azure: 
[PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6191)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
This is an automated message from the 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-unsubscr...@hudi.apache.org

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




[hudi] branch master updated: [HUDI-2189] Adding delete partitions support to DeltaStreamer (#4787)

2022-02-21 Thread sivabalan
This is an automated email from the ASF dual-hosted git repository.

sivabalan pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hudi.git


The following commit(s) were added to refs/heads/master by this push:
 new 14dbbdf  [HUDI-2189] Adding delete partitions support to DeltaStreamer 
(#4787)
14dbbdf is described below

commit 14dbbdf4c7a45ab5a10889f8e558984455315829
Author: Sivabalan Narayanan 
AuthorDate: Tue Feb 22 00:01:30 2022 -0500

[HUDI-2189] Adding delete partitions support to DeltaStreamer (#4787)
---
 .../hudi/utilities/deltastreamer/DeltaSync.java|  4 ++
 .../functional/HoodieDeltaStreamerTestBase.java|  2 +-
 .../functional/TestHoodieDeltaStreamer.java| 57 +-
 3 files changed, 61 insertions(+), 2 deletions(-)

diff --git 
a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java
 
b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java
index c376243..082a9b1 100644
--- 
a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java
+++ 
b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java
@@ -585,6 +585,10 @@ public class DeltaSync implements Serializable {
   case INSERT_OVERWRITE_TABLE:
 writeStatusRDD = writeClient.insertOverwriteTable(records, 
instantTime).getWriteStatuses();
 break;
+  case DELETE_PARTITION:
+List partitions = records.map(record -> 
record.getPartitionPath()).distinct().collect();
+writeStatusRDD = writeClient.deletePartitions(partitions, 
instantTime).getWriteStatuses();
+break;
   default:
 throw new HoodieDeltaStreamerException("Unknown operation : " + 
cfg.operation);
 }
diff --git 
a/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/HoodieDeltaStreamerTestBase.java
 
b/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/HoodieDeltaStreamerTestBase.java
index 9b7ee3b..02b1848 100644
--- 
a/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/HoodieDeltaStreamerTestBase.java
+++ 
b/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/HoodieDeltaStreamerTestBase.java
@@ -173,7 +173,7 @@ public class HoodieDeltaStreamerTestBase extends 
UtilitiesTestBase {
 props.setProperty("include", "sql-transformer.properties");
 props.setProperty("hoodie.datasource.write.keygenerator.class", 
TestHoodieDeltaStreamer.TestGenerator.class.getName());
 props.setProperty("hoodie.datasource.write.recordkey.field", "_row_key");
-props.setProperty("hoodie.datasource.write.partitionpath.field", 
"not_there");
+props.setProperty("hoodie.datasource.write.partitionpath.field", 
"partition_path");
 
props.setProperty("hoodie.deltastreamer.schemaprovider.source.schema.file", 
dfsBasePath + "/source.avsc");
 
props.setProperty("hoodie.deltastreamer.schemaprovider.target.schema.file", 
dfsBasePath + "/target.avsc");
 
diff --git 
a/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamer.java
 
b/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamer.java
index a6fdf00..1c80896 100644
--- 
a/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamer.java
+++ 
b/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamer.java
@@ -272,6 +272,19 @@ public class TestHoodieDeltaStreamer extends 
HoodieDeltaStreamerTestBase {
   assertEquals(expected, recordCount);
 }
 
+static Map getPartitionRecordCount(String basePath, 
SQLContext sqlContext) {
+  sqlContext.clearCache();
+  List rows = 
sqlContext.read().format("org.apache.hudi").load(basePath).groupBy(HoodieRecord.PARTITION_PATH_METADATA_FIELD).count().collectAsList();
+  Map partitionRecordCount = new HashMap<>();
+  rows.stream().forEach(row -> partitionRecordCount.put(row.getString(0), 
row.getLong(1)));
+  return partitionRecordCount;
+}
+
+static void assertNoPartitionMatch(String basePath, SQLContext sqlContext, 
String partitionToValidate) {
+  sqlContext.clearCache();
+  assertEquals(0, 
sqlContext.read().format("org.apache.hudi").load(basePath).filter(HoodieRecord.PARTITION_PATH_METADATA_FIELD
 + " = " + partitionToValidate).count());
+}
+
 static void assertDistinctRecordCount(long expected, String tablePath, 
SQLContext sqlContext) {
   sqlContext.clearCache();
   long recordCount = 
sqlContext.read().format("org.apache.hudi").load(tablePath).select("_hoodie_record_key").distinct().count();
@@ -1378,6 +1391,13 @@ public class TestHoodieDeltaStreamer extends 
HoodieDeltaStreamerTestBase {
 
   private void prepareParquetDFSSource(boolean useSchemaProvider, boolean 
hasTransformer, String sourceSchemaFile, String targetSchemaFile,
String propsFileName, String 
parquetSourceRoot

[GitHub] [hudi] nsivabalan merged pull request #4787: [HUDI-2189] Adding delete partitions support to DeltaStreamer

2022-02-21 Thread GitBox


nsivabalan merged pull request #4787:
URL: https://github.com/apache/hudi/pull/4787


   


-- 
This is an automated message from the 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-unsubscr...@hudi.apache.org

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




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

2022-02-21 Thread GitBox


hudi-bot commented on pull request #4679:
URL: https://github.com/apache/hudi/pull/4679#issuecomment-1047421762


   
   ## CI report:
   
   * 48b162c6c411b9cb5b605d006acc8d6e7d45cec8 Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6189)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
This is an automated message from the 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-unsubscr...@hudi.apache.org

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




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

2022-02-21 Thread GitBox


hudi-bot removed a comment on pull request #4679:
URL: https://github.com/apache/hudi/pull/4679#issuecomment-1047383322


   
   ## 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)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
This is an automated message from the 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-unsubscr...@hudi.apache.org

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




[GitHub] [hudi] nsivabalan commented on issue #4604: [SUPPORT] Archive functionality fails

2022-02-21 Thread GitBox


nsivabalan commented on issue #4604:
URL: https://github.com/apache/hudi/issues/4604#issuecomment-1047411843


   with 0.9.0, if you disable metadata, may I know if you are still seeing NPE 
w/ archival. 


-- 
This is an automated message from the 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-unsubscr...@hudi.apache.org

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




[jira] [Updated] (HUDI-3441) Clean up markers of completed commits in .hoodie/.temp folder

2022-02-21 Thread Ethan Guo (Jira)


 [ 
https://issues.apache.org/jira/browse/HUDI-3441?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Ethan Guo updated HUDI-3441:

Issue Type: Task  (was: Improvement)

> Clean up markers of completed commits in .hoodie/.temp folder
> -
>
> Key: HUDI-3441
> URL: https://issues.apache.org/jira/browse/HUDI-3441
> Project: Apache Hudi
>  Issue Type: Task
>  Components: writer-core
>Reporter: Ethan Guo
>Priority: Blocker
> Fix For: 0.11.0
>
>




--
This message was sent by Atlassian Jira
(v8.20.1#820001)


[jira] [Updated] (HUDI-3467) Check shutdown logic with async compaction in Spark Structured Streaming

2022-02-21 Thread Ethan Guo (Jira)


 [ 
https://issues.apache.org/jira/browse/HUDI-3467?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Ethan Guo updated HUDI-3467:

Story Points: 3

> Check shutdown logic with async compaction in Spark Structured Streaming
> 
>
> Key: HUDI-3467
> URL: https://issues.apache.org/jira/browse/HUDI-3467
> Project: Apache Hudi
>  Issue Type: Bug
>Reporter: Ethan Guo
>Priority: Blocker
> Fix For: 0.11.0
>
>




--
This message was sent by Atlassian Jira
(v8.20.1#820001)


[jira] [Updated] (HUDI-3467) Check shutdown logic with async compaction in Spark Structured Streaming

2022-02-21 Thread Ethan Guo (Jira)


 [ 
https://issues.apache.org/jira/browse/HUDI-3467?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Ethan Guo updated HUDI-3467:

Issue Type: Bug  (was: Improvement)

> Check shutdown logic with async compaction in Spark Structured Streaming
> 
>
> Key: HUDI-3467
> URL: https://issues.apache.org/jira/browse/HUDI-3467
> Project: Apache Hudi
>  Issue Type: Bug
>Reporter: Ethan Guo
>Priority: Blocker
> Fix For: 0.11.0
>
>




--
This message was sent by Atlassian Jira
(v8.20.1#820001)


[jira] [Updated] (HUDI-3449) Async compaction cannot proceed due to archived deltacommit in metadata table

2022-02-21 Thread Ethan Guo (Jira)


 [ 
https://issues.apache.org/jira/browse/HUDI-3449?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Ethan Guo updated HUDI-3449:

Sprint: Hudi-Sprint-Feb-14

> Async compaction cannot proceed due to archived deltacommit in metadata table
> -
>
> Key: HUDI-3449
> URL: https://issues.apache.org/jira/browse/HUDI-3449
> Project: Apache Hudi
>  Issue Type: Bug
>  Components: writer-core
>Reporter: Ethan Guo
>Assignee: Ethan Guo
>Priority: Blocker
> Fix For: 0.11.0
>
>
> Environment: deltastreamer continuous mode writing MOR table, async 
> compaction and clean enabled.  Metadata table enabled.
> Repeatly hitting the following issue after the deltastreamer is running for 
> some time:
> {code:java}
> org.apache.hudi.exception.HoodieMetadataException: The instant 
> [20220216224839969__deltacommit__COMPLETED] required to sync rollback of 
> 20220216224839969 has been archived
>     at 
> org.apache.hudi.metadata.HoodieTableMetadataUtil.lambda$processRollbackMetadata$15(HoodieTableMetadataUtil.java:521)
>     at java.util.HashMap$Values.forEach(HashMap.java:981)
>     at 
> java.util.Collections$UnmodifiableCollection.forEach(Collections.java:1082)
>     at 
> org.apache.hudi.metadata.HoodieTableMetadataUtil.processRollbackMetadata(HoodieTableMetadataUtil.java:498)
>     at 
> org.apache.hudi.metadata.HoodieTableMetadataUtil.convertMetadataToRollbackRecords(HoodieTableMetadataUtil.java:472)
>     at 
> org.apache.hudi.metadata.HoodieTableMetadataUtil.convertMetadataToRecords(HoodieTableMetadataUtil.java:437)
>     at 
> org.apache.hudi.metadata.HoodieBackedTableMetadataWriter.update(HoodieBackedTableMetadataWriter.java:726)
>     at 
> org.apache.hudi.table.action.BaseActionExecutor.lambda$writeTableMetadata$2(BaseActionExecutor.java:77)
>     at org.apache.hudi.common.util.Option.ifPresent(Option.java:96)
>     at 
> org.apache.hudi.table.action.BaseActionExecutor.writeTableMetadata(BaseActionExecutor.java:77)
>     at 
> org.apache.hudi.table.action.rollback.BaseRollbackActionExecutor.finishRollback(BaseRollbackActionExecutor.java:244)
>     at 
> org.apache.hudi.table.action.rollback.BaseRollbackActionExecutor.runRollback(BaseRollbackActionExecutor.java:122)
>     at 
> org.apache.hudi.table.action.rollback.BaseRollbackActionExecutor.execute(BaseRollbackActionExecutor.java:144)
>     at 
> org.apache.hudi.table.HoodieSparkMergeOnReadTable.rollback(HoodieSparkMergeOnReadTable.java:172)
>     at 
> org.apache.hudi.table.HoodieTable.rollbackInflightCompaction(HoodieTable.java:524)
>     at 
> org.apache.hudi.client.SparkRDDWriteClient.compact(SparkRDDWriteClient.java:342)
>     at 
> org.apache.hudi.client.BaseHoodieWriteClient.compact(BaseHoodieWriteClient.java:919)
>     at 
> org.apache.hudi.client.HoodieSparkCompactor.compact(HoodieSparkCompactor.java:52)
>     at 
> org.apache.hudi.async.AsyncCompactService.lambda$null$1(AsyncCompactService.java:89)
>     at 
> java.util.concurrent.CompletableFuture$AsyncSupply.run(CompletableFuture.java:1604)
>     at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
>     at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
>     at java.lang.Thread.run(Thread.java:748) {code}



--
This message was sent by Atlassian Jira
(v8.20.1#820001)


[jira] [Updated] (HUDI-3443) Async clustering cannot proceed due to archived deltacommit in metadata table

2022-02-21 Thread Ethan Guo (Jira)


 [ 
https://issues.apache.org/jira/browse/HUDI-3443?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Ethan Guo updated HUDI-3443:

Sprint: Hudi-Sprint-Feb-14

> Async clustering cannot proceed due to archived deltacommit in metadata table
> -
>
> Key: HUDI-3443
> URL: https://issues.apache.org/jira/browse/HUDI-3443
> Project: Apache Hudi
>  Issue Type: Bug
>Reporter: Ethan Guo
>Assignee: Ethan Guo
>Priority: Blocker
> Fix For: 0.11.0
>
>
> Environment: deltastreamer continuous mode writing COW table, async 
> clustering and clean enabled.  Metadata table enabled.
> Repeatly hitting the following issue after the deltastreamer is running for 
> some time:
> {code:java}
> org.apache.hudi.exception.HoodieMetadataException: The instant 
> [20220209121825078__deltacommit__COMPLETED] required to sync rollback of 
> 20220209121825078 has been archived
>     at 
> org.apache.hudi.metadata.HoodieTableMetadataUtil.lambda$processRollbackMetadata$15(HoodieTableMetadataUtil.java:521)
>     at java.util.HashMap$Values.forEach(HashMap.java:981)
>     at 
> java.util.Collections$UnmodifiableCollection.forEach(Collections.java:1082)
>     at 
> org.apache.hudi.metadata.HoodieTableMetadataUtil.processRollbackMetadata(HoodieTableMetadataUtil.java:498)
>     at 
> org.apache.hudi.metadata.HoodieTableMetadataUtil.convertMetadataToRollbackRecords(HoodieTableMetadataUtil.java:472)
>     at 
> org.apache.hudi.metadata.HoodieTableMetadataUtil.convertMetadataToRecords(HoodieTableMetadataUtil.java:437)
>     at 
> org.apache.hudi.metadata.HoodieBackedTableMetadataWriter.update(HoodieBackedTableMetadataWriter.java:726)
>     at 
> org.apache.hudi.table.action.BaseActionExecutor.lambda$writeTableMetadata$2(BaseActionExecutor.java:77)
>     at org.apache.hudi.common.util.Option.ifPresent(Option.java:96)
>     at 
> org.apache.hudi.table.action.BaseActionExecutor.writeTableMetadata(BaseActionExecutor.java:77)
>     at 
> org.apache.hudi.table.action.rollback.BaseRollbackActionExecutor.finishRollback(BaseRollbackActionExecutor.java:244)
>     at 
> org.apache.hudi.table.action.rollback.BaseRollbackActionExecutor.runRollback(BaseRollbackActionExecutor.java:122)
>     at 
> org.apache.hudi.table.action.rollback.BaseRollbackActionExecutor.execute(BaseRollbackActionExecutor.java:144)
>     at 
> org.apache.hudi.table.HoodieSparkCopyOnWriteTable.rollback(HoodieSparkCopyOnWriteTable.java:346)
>     at 
> org.apache.hudi.client.BaseHoodieWriteClient.rollbackInflightClustering(BaseHoodieWriteClient.java:1211)
>     at 
> org.apache.hudi.client.SparkRDDWriteClient.cluster(SparkRDDWriteClient.java:361)
>     at 
> org.apache.hudi.client.HoodieSparkClusteringClient.cluster(HoodieSparkClusteringClient.java:54)
>     at 
> org.apache.hudi.async.AsyncClusteringService.lambda$null$1(AsyncClusteringService.java:79)
>     at 
> java.util.concurrent.CompletableFuture$AsyncSupply.run(CompletableFuture.java:1604)
>     at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
>     at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
>     at java.lang.Thread.run(Thread.java:748) {code}



--
This message was sent by Atlassian Jira
(v8.20.1#820001)


[jira] [Updated] (HUDI-2606) Enable metadata by default for readers

2022-02-21 Thread Ethan Guo (Jira)


 [ 
https://issues.apache.org/jira/browse/HUDI-2606?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Ethan Guo updated HUDI-2606:

Sprint: Hudi-Sprint-Feb-14

> Enable metadata by default for readers
> --
>
> Key: HUDI-2606
> URL: https://issues.apache.org/jira/browse/HUDI-2606
> Project: Apache Hudi
>  Issue Type: Task
>Reporter: sivabalan narayanan
>Assignee: Ethan Guo
>Priority: Blocker
>  Labels: pull-request-available
> Fix For: 0.11.0
>
>




--
This message was sent by Atlassian Jira
(v8.20.1#820001)


[jira] [Closed] (HUDI-2602) RFC: Metadata based range index

2022-02-21 Thread Alexey Kudinkin (Jira)


 [ 
https://issues.apache.org/jira/browse/HUDI-2602?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Alexey Kudinkin closed HUDI-2602.
-
Resolution: Fixed

> RFC: Metadata based range index
> ---
>
> Key: HUDI-2602
> URL: https://issues.apache.org/jira/browse/HUDI-2602
> Project: Apache Hudi
>  Issue Type: Task
>Reporter: sivabalan narayanan
>Assignee: Manoj Govindassamy
>Priority: Blocker
> Fix For: 0.10.0
>
>
> Publish design doc/RFC for metadata based range index



--
This message was sent by Atlassian Jira
(v8.20.1#820001)


[GitHub] [hudi] hudi-bot removed a comment on pull request #4861: [HUDI-3461] The archived timeline for flink streaming reader should n…

2022-02-21 Thread GitBox


hudi-bot removed a comment on pull request #4861:
URL: https://github.com/apache/hudi/pull/4861#issuecomment-1047402591


   
   ## CI report:
   
   * 805c912d164c10de9db27434d8efc7890250f3eb Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6163)
 
   * 75f70ba1856029f7530b75cbd314597af55db867 UNKNOWN
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
This is an automated message from the 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-unsubscr...@hudi.apache.org

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




[GitHub] [hudi] hudi-bot commented on pull request #4861: [HUDI-3461] The archived timeline for flink streaming reader should n…

2022-02-21 Thread GitBox


hudi-bot commented on pull request #4861:
URL: https://github.com/apache/hudi/pull/4861#issuecomment-1047404397


   
   ## CI report:
   
   * 805c912d164c10de9db27434d8efc7890250f3eb Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6163)
 
   * 75f70ba1856029f7530b75cbd314597af55db867 Azure: 
[PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6191)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
This is an automated message from the 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-unsubscr...@hudi.apache.org

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




[GitHub] [hudi] hudi-bot removed a comment on pull request #4861: [HUDI-3461] The archived timeline for flink streaming reader should n…

2022-02-21 Thread GitBox


hudi-bot removed a comment on pull request #4861:
URL: https://github.com/apache/hudi/pull/4861#issuecomment-1046801198


   
   ## CI report:
   
   * 805c912d164c10de9db27434d8efc7890250f3eb Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6163)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
This is an automated message from the 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-unsubscr...@hudi.apache.org

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




[GitHub] [hudi] hudi-bot commented on pull request #4861: [HUDI-3461] The archived timeline for flink streaming reader should n…

2022-02-21 Thread GitBox


hudi-bot commented on pull request #4861:
URL: https://github.com/apache/hudi/pull/4861#issuecomment-1047402591


   
   ## CI report:
   
   * 805c912d164c10de9db27434d8efc7890250f3eb Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6163)
 
   * 75f70ba1856029f7530b75cbd314597af55db867 UNKNOWN
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
This is an automated message from the 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-unsubscr...@hudi.apache.org

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




[jira] [Assigned] (HUDI-1292) [Umbrella] RFC-15 : Metadata Table for File Listing and other table metadata

2022-02-21 Thread Ethan Guo (Jira)


 [ 
https://issues.apache.org/jira/browse/HUDI-1292?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Ethan Guo reassigned HUDI-1292:
---

Assignee: Ethan Guo  (was: Prashant Wason)

> [Umbrella] RFC-15 : Metadata Table for File Listing and other table metadata
> 
>
> Key: HUDI-1292
> URL: https://issues.apache.org/jira/browse/HUDI-1292
> Project: Apache Hudi
>  Issue Type: Epic
>  Components: spark, writer-core
>Affects Versions: 0.9.0
>Reporter: Vinoth Chandar
>Assignee: Ethan Guo
>Priority: Blocker
>  Labels: hudi-umbrellas, pull-request-available
> Fix For: 0.11.0
>
>
> This is the umbrella ticket that tracks the overall implementation of RFC-15



--
This message was sent by Atlassian Jira
(v8.20.1#820001)


[GitHub] [hudi] hudi-bot commented on pull request #4848: [HUDI-3356][HUDI-3203] HoodieData for metadata index records, bloom and colstats init

2022-02-21 Thread GitBox


hudi-bot commented on pull request #4848:
URL: https://github.com/apache/hudi/pull/4848#issuecomment-1047401380


   
   ## CI report:
   
   * 97f253e3d9ef2c8caf05810d42e5f54e7598d4de Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6187)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
This is an automated message from the 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-unsubscr...@hudi.apache.org

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




[GitHub] [hudi] hudi-bot removed a comment on pull request #4848: [HUDI-3356][HUDI-3203] HoodieData for metadata index records, bloom and colstats init

2022-02-21 Thread GitBox


hudi-bot removed a comment on pull request #4848:
URL: https://github.com/apache/hudi/pull/4848#issuecomment-1047369471


   
   ## CI report:
   
   * 63aac434acd15223dc186635f963e97367e9 Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6124)
 
   * 97f253e3d9ef2c8caf05810d42e5f54e7598d4de Azure: 
[PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6187)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
This is an automated message from the 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-unsubscr...@hudi.apache.org

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




[GitHub] [hudi] nsivabalan commented on issue #3731: [SUPPORT] Concurrent write (OCC) on distinct partitions random errors

2022-02-21 Thread GitBox


nsivabalan commented on issue #3731:
URL: https://github.com/apache/hudi/issues/3731#issuecomment-1047401320


   @parisni : if you were able to get it resolved, feel free to close out the 
issue. if not, let us know. if we triage any bugs, we can try to fix it once 
for all in 0.11 and solidify multi writers for hudi. 


-- 
This is an automated message from the 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-unsubscr...@hudi.apache.org

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




[GitHub] [hudi] nsivabalan commented on issue #3868: [SUPPORT] hive syncing with `--spark-datasource` (first title was: Querying hudi datasets from standalone metastore)

2022-02-21 Thread GitBox


nsivabalan commented on issue #3868:
URL: https://github.com/apache/hudi/issues/3868#issuecomment-1047400875


   @matthiasdg : can we have any updates here please. 


-- 
This is an automated message from the 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-unsubscr...@hudi.apache.org

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




[GitHub] [hudi] nsivabalan commented on issue #3965: [SUPPORT] Upserts result in truncated milliseconds from timestamps

2022-02-21 Thread GitBox


nsivabalan commented on issue #3965:
URL: https://github.com/apache/hudi/issues/3965#issuecomment-1047400781


   Closing this due to no activity for the past 3+ months. 


-- 
This is an automated message from the 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-unsubscr...@hudi.apache.org

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




[GitHub] [hudi] nsivabalan closed issue #3965: [SUPPORT] Upserts result in truncated milliseconds from timestamps

2022-02-21 Thread GitBox


nsivabalan closed issue #3965:
URL: https://github.com/apache/hudi/issues/3965


   


-- 
This is an automated message from the 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-unsubscr...@hudi.apache.org

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




[GitHub] [hudi] nsivabalan commented on issue #4456: [SUPPORT] MultiWriter w/ DynamoDB - Unable to acquire lock, lock object null

2022-02-21 Thread GitBox


nsivabalan commented on issue #4456:
URL: https://github.com/apache/hudi/issues/4456#issuecomment-1047400181


   @mainamit : let us know if you were able to get it working. Feel free to 
close out the github issue. If you are still facing the issue, do ping w/ more 
details. Wenning should be able to assist you. 


-- 
This is an automated message from the 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-unsubscr...@hudi.apache.org

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




[GitHub] [hudi] nsivabalan commented on issue #4506: [SUPPORT] Hive Sync fails silently with embedded derby hive metastore

2022-02-21 Thread GitBox


nsivabalan commented on issue #4506:
URL: https://github.com/apache/hudi/issues/4506#issuecomment-1047399870


   Hey @parisni : can you share the logs. If you got it resolved, feel free to 
close the issue. Or would be nice if you can share more logs for Sagar to 
triage. 


-- 
This is an automated message from the 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-unsubscr...@hudi.apache.org

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




[GitHub] [hudi] nsivabalan commented on issue #4622: [SUPPORT] Can't query Redshift rows even after downgrade from 0.10

2022-02-21 Thread GitBox


nsivabalan commented on issue #4622:
URL: https://github.com/apache/hudi/issues/4622#issuecomment-1047399531


   yes, makes sense. I will see where we can document 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-unsubscr...@hudi.apache.org

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




[GitHub] [hudi] nsivabalan commented on issue #4635: [SUPPORT] Bulk write failing due to hudi timeline archive exception

2022-02-21 Thread GitBox


nsivabalan commented on issue #4635:
URL: https://github.com/apache/hudi/issues/4635#issuecomment-1047399253


   @VIKASPATID : is it possible to give us reproducible steps. would help us 
triage it faster. 


-- 
This is an automated message from the 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-unsubscr...@hudi.apache.org

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




[GitHub] [hudi] nsivabalan commented on issue #4683: [SUPPORT] Hive ro table read error

2022-02-21 Thread GitBox


nsivabalan commented on issue #4683:
URL: https://github.com/apache/hudi/issues/4683#issuecomment-1047398703


   @waywtdcc : if you got unblocked or resolved the issue, feel free to close 
the github issue. Or do furnish more details/updates. 


-- 
This is an automated message from the 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-unsubscr...@hudi.apache.org

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




[GitHub] [hudi] nsivabalan commented on issue #4700: [SUPPORT] Adding new column to table is not propagated to Hive via HMS sync mode

2022-02-21 Thread GitBox


nsivabalan commented on issue #4700:
URL: https://github.com/apache/hudi/issues/4700#issuecomment-1047397843


   @xiarixiaoyao : if issue has been taken care, can we close it out. Or if you 
have a tracking jira, feel free to close it out as well. 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-unsubscr...@hudi.apache.org

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




[hudi] branch master updated (0dee8ed -> 7e1ea06)

2022-02-21 Thread xushiyan
This is an automated email from the ASF dual-hosted git repository.

xushiyan pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/hudi.git.


from 0dee8ed  [HUDI-2925] Fix duplicate cleaning of same files when 
unfinished clean operations are present using a config. (#4212)
 add 7e1ea06  [MINOR] Fix typos and improve docs in HoodieMetadataConfig 
(#4867)

No new revisions were added by this update.

Summary of changes:
 .../hudi/common/config/HoodieMetadataConfig.java   | 33 +-
 1 file changed, 19 insertions(+), 14 deletions(-)


[GitHub] [hudi] xushiyan merged pull request #4867: [MINOR] Fix typos and improve docs in HoodieMetadataConfig

2022-02-21 Thread GitBox


xushiyan merged pull request #4867:
URL: https://github.com/apache/hudi/pull/4867


   


-- 
This is an automated message from the 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-unsubscr...@hudi.apache.org

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




[GitHub] [hudi] YannByron commented on issue #4857: [SUPPORT]Failed to create a table

2022-02-21 Thread GitBox


YannByron commented on issue #4857:
URL: https://github.com/apache/hudi/issues/4857#issuecomment-1047391075


   @melin you need also to set `spark.sql.catalog.spark_catalog` to 
`org.apache.spark.sql.hudi.catalog.HoodieCatalog` before you open a spark 
session, 'cause that the master branch has supported spark DatasourceV2.


-- 
This is an automated message from the 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-unsubscr...@hudi.apache.org

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




[GitHub] [hudi] hudi-bot removed a comment on pull request #4867: [MINOR] Fix typos and improve docs in HoodieMetadataConfig

2022-02-21 Thread GitBox


hudi-bot removed a comment on pull request #4867:
URL: https://github.com/apache/hudi/pull/4867#issuecomment-1047347224


   
   ## CI report:
   
   * c6c5f42daf63c57adc429159c588bcd6796b77c0 Azure: 
[PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6186)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
This is an automated message from the 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-unsubscr...@hudi.apache.org

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




[GitHub] [hudi] hudi-bot commented on pull request #4867: [MINOR] Fix typos and improve docs in HoodieMetadataConfig

2022-02-21 Thread GitBox


hudi-bot commented on pull request #4867:
URL: https://github.com/apache/hudi/pull/4867#issuecomment-1047389200


   
   ## CI report:
   
   * c6c5f42daf63c57adc429159c588bcd6796b77c0 Azure: 
[SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6186)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
This is an automated message from the 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-unsubscr...@hudi.apache.org

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




[GitHub] [hudi] waywtdcc commented on issue #4858: [SUPPORT] data loss occurred flink hudi

2022-02-21 Thread GitBox


waywtdcc commented on issue #4858:
URL: https://github.com/apache/hudi/issues/4858#issuecomment-1047385719


   > Did you use the MOR table and the source is a CDC changelog ? This is a 
known bug and i have created an issue before: 
https://issues.apache.org/jira/browse/HUDI-2752, we would solve that in release 
0.11.0, for the current solution, you can open the changelog mode with option 
`changelog.enabled`, or use the COW table type.
   
   Does cow mode support near real-time writing?


-- 
This is an automated message from the 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-unsubscr...@hudi.apache.org

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




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

2022-02-21 Thread GitBox


hudi-bot commented on pull request #4679:
URL: https://github.com/apache/hudi/pull/4679#issuecomment-1047383322


   
   ## 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)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
This is an automated message from the 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-unsubscr...@hudi.apache.org

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




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

2022-02-21 Thread GitBox


hudi-bot removed a comment on pull request #4679:
URL: https://github.com/apache/hudi/pull/4679#issuecomment-1047381744


   
   ## CI report:
   
   *  Unknown: [CANCELED](TBD) 
   * 48b162c6c411b9cb5b605d006acc8d6e7d45cec8 UNKNOWN
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
This is an automated message from the 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-unsubscr...@hudi.apache.org

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




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

2022-02-21 Thread GitBox


hudi-bot commented on pull request #4679:
URL: https://github.com/apache/hudi/pull/4679#issuecomment-1047381744


   
   ## CI report:
   
   *  Unknown: [CANCELED](TBD) 
   * 48b162c6c411b9cb5b605d006acc8d6e7d45cec8 UNKNOWN
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
This is an automated message from the 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-unsubscr...@hudi.apache.org

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




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

2022-02-21 Thread GitBox


hudi-bot removed a comment on pull request #4679:
URL: https://github.com/apache/hudi/pull/4679#issuecomment-1041435375


   
   ## CI report:
   
   * ddbf502d11717d800971f736fa81054afa1873b7 Azure: 
[SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6056)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
This is an automated message from the 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-unsubscr...@hudi.apache.org

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




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

2022-02-21 Thread GitBox


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-unsubscr...@hudi.apache.org

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




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

2022-02-21 Thread GitBox


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-unsubscr...@hudi.apache.org

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




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

2022-02-21 Thread GitBox


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-unsubscr...@hudi.apache.org

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




[GitHub] [hudi] hudi-bot removed a comment on pull request #4787: [HUDI-2189] Adding delete partitions support to DeltaStreamer

2022-02-21 Thread GitBox


hudi-bot removed a comment on pull request #4787:
URL: https://github.com/apache/hudi/pull/4787#issuecomment-1047342399


   
   ## CI report:
   
   * f288ae1ff5f7a01abb9a320acf448079c44956ce Azure: 
[SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5883)
 
   * 693f86e35cea5c472a54c5f8b1fab5dfa25bb06a Azure: 
[PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6185)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
This is an automated message from the 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-unsubscr...@hudi.apache.org

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




[GitHub] [hudi] hudi-bot commented on pull request #4787: [HUDI-2189] Adding delete partitions support to DeltaStreamer

2022-02-21 Thread GitBox


hudi-bot commented on pull request #4787:
URL: https://github.com/apache/hudi/pull/4787#issuecomment-1047378621


   
   ## CI report:
   
   * 693f86e35cea5c472a54c5f8b1fab5dfa25bb06a Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6185)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
This is an automated message from the 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-unsubscr...@hudi.apache.org

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




[jira] [Assigned] (HUDI-1517) Create marker file for every log file

2022-02-21 Thread sivabalan narayanan (Jira)


 [ 
https://issues.apache.org/jira/browse/HUDI-1517?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

sivabalan narayanan reassigned HUDI-1517:
-

Assignee: ZiyueGuan

> Create marker file for every log file
> -
>
> Key: HUDI-1517
> URL: https://issues.apache.org/jira/browse/HUDI-1517
> Project: Apache Hudi
>  Issue Type: Improvement
>  Components: writer-core
>Reporter: sivabalan narayanan
>Assignee: ZiyueGuan
>Priority: Major
>
> As of now, hudi creates marker file based on base file. But we might need to 
> fix this to create one marker file per log file denoting the actual log file 
> info. We can leverage this during metadata sync w/ rollback and restore 
> rather than doing a file listing. 



--
This message was sent by Atlassian Jira
(v8.20.1#820001)


[jira] [Commented] (HUDI-1517) Create marker file for every log file

2022-02-21 Thread sivabalan narayanan (Jira)


[ 
https://issues.apache.org/jira/browse/HUDI-1517?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17495834#comment-17495834
 ] 

sivabalan narayanan commented on HUDI-1517:
---

[~guanziyue] Please go ahead and grab the ticket if you are interested.  

> Create marker file for every log file
> -
>
> Key: HUDI-1517
> URL: https://issues.apache.org/jira/browse/HUDI-1517
> Project: Apache Hudi
>  Issue Type: Improvement
>  Components: writer-core
>Reporter: sivabalan narayanan
>Priority: Major
>
> As of now, hudi creates marker file based on base file. But we might need to 
> fix this to create one marker file per log file denoting the actual log file 
> info. We can leverage this during metadata sync w/ rollback and restore 
> rather than doing a file listing. 



--
This message was sent by Atlassian Jira
(v8.20.1#820001)


[hudi] branch master updated: [HUDI-2925] Fix duplicate cleaning of same files when unfinished clean operations are present using a config. (#4212)

2022-02-21 Thread sivabalan
This is an automated email from the ASF dual-hosted git repository.

sivabalan pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hudi.git


The following commit(s) were added to refs/heads/master by this push:
 new 0dee8ed  [HUDI-2925] Fix duplicate cleaning of same files when 
unfinished clean operations are present using a config. (#4212)
0dee8ed is described below

commit 0dee8edc9741ee99e1e2bf98efd9673003fcb1e7
Author: Prashant Wason 
AuthorDate: Mon Feb 21 18:53:03 2022 -0800

[HUDI-2925] Fix duplicate cleaning of same files when unfinished clean 
operations are present using a config. (#4212)


Co-authored-by: sivabalan 
---
 .../apache/hudi/client/BaseHoodieWriteClient.java  | 31 ++
 .../apache/hudi/config/HoodieCompactionConfig.java | 12 
 .../org/apache/hudi/config/HoodieWriteConfig.java  |  4 ++
 .../hudi/table/action/BaseActionExecutor.java  |  2 +-
 .../table/action/clean/CleanActionExecutor.java|  8 ++-
 .../java/org/apache/hudi/table/TestCleaner.java| 68 ++
 .../org/apache/hudi/common/util/CleanerUtils.java  |  8 +++
 7 files changed, 118 insertions(+), 15 deletions(-)

diff --git 
a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java
 
b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java
index f3dc53b..7b67ff5 100644
--- 
a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java
+++ 
b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java
@@ -767,21 +767,28 @@ public abstract class BaseHoodieWriteClient rollbackFailedWrites(skipLocking));
-HoodieCleanMetadata metadata = createTable(config, 
hadoopConf).clean(context, cleanInstantTime, skipLocking);
-if (timerContext != null && metadata != null) {
-  long durationMs = metrics.getDurationInMs(timerContext.stop());
-  metrics.updateCleanMetrics(durationMs, metadata.getTotalFilesDeleted());
-  LOG.info("Cleaned " + metadata.getTotalFilesDeleted() + " files"
-  + " Earliest Retained Instant :" + 
metadata.getEarliestCommitToRetain()
-  + " cleanerElapsedMs" + durationMs);
+
+HoodieCleanMetadata metadata = null;
+HoodieTable table = createTable(config, hadoopConf);
+if (config.allowMultipleCleans() || 
!table.getActiveTimeline().getCleanerTimeline().filterInflightsAndRequested().firstInstant().isPresent())
 {
+  LOG.info("Cleaner started");
+  // proceed only if multiple clean schedules are enabled or if there are 
no pending cleans.
+  if (scheduleInline) {
+scheduleTableServiceInternal(cleanInstantTime, Option.empty(), 
TableServiceType.CLEAN);
+table.getMetaClient().reloadActiveTimeline();
+  }
+
+  metadata = table.clean(context, cleanInstantTime, skipLocking);
+  if (timerContext != null && metadata != null) {
+long durationMs = metrics.getDurationInMs(timerContext.stop());
+metrics.updateCleanMetrics(durationMs, 
metadata.getTotalFilesDeleted());
+LOG.info("Cleaned " + metadata.getTotalFilesDeleted() + " files"
++ " Earliest Retained Instant :" + 
metadata.getEarliestCommitToRetain()
++ " cleanerElapsedMs" + durationMs);
+  }
 }
 return metadata;
   }
diff --git 
a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieCompactionConfig.java
 
b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieCompactionConfig.java
index 130d379..0aac930 100644
--- 
a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieCompactionConfig.java
+++ 
b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieCompactionConfig.java
@@ -278,6 +278,13 @@ public class HoodieCompactionConfig extends HoodieConfig {
   .withDocumentation("The average record size. If not explicitly 
specified, hudi will compute the "
   + "record size estimate compute dynamically based on commit 
metadata. "
   + " This is critical in computing the insert parallelism and 
bin-packing inserts into small files.");
+  
+  public static final ConfigProperty ALLOW_MULTIPLE_CLEANS = 
ConfigProperty
+  .key("hoodie.clean.allow.multiple")
+  .defaultValue(true)
+  .sinceVersion("0.11.0")
+  .withDocumentation("Allows scheduling/executing multiple cleans by 
enabling this config. If users prefer to strictly ensure clean requests should 
be mutually exclusive, "
+  + ".i.e. a 2nd clean will not be scheduled if another clean is not 
yet completed to avoid repeat cleaning of same files, they might want to 
disable this config.");
 
   public static final ConfigProperty ARCHIVE_MERGE_FILES_BATCH_SIZE = 
ConfigProperty
   .key("hoodie.archive.merge.files.batch.size")
@@ -642,6 +649,11 @@ public class HoodieCompactionConfig extends HoodieConfig {
   return this;
 

[GitHub] [hudi] nsivabalan merged pull request #4212: [HUDI-2925] Fix duplicate cleaning of same files when unfinished clean operations are present.

2022-02-21 Thread GitBox


nsivabalan merged pull request #4212:
URL: https://github.com/apache/hudi/pull/4212


   


-- 
This is an automated message from the 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-unsubscr...@hudi.apache.org

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




[GitHub] [hudi] danny0405 commented on a change in pull request #4861: [HUDI-3461] The archived timeline for flink streaming reader should n…

2022-02-21 Thread GitBox


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



##
File path: 
hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieArchivedTimeline.java
##
@@ -117,12 +130,19 @@ public static Path getArchiveLogPath(String 
archiveFolder) {
   }
 
   public void loadInstantDetailsInMemory(String startTs, String endTs) {
-loadInstants(startTs, endTs);
+setInstants(loadInstants(startTs, endTs));
   }
 
   public void loadCompletedInstantDetailsInMemory() {
-loadInstants(null, true,
+final List instants = loadInstants(null, true,
 record -> 
HoodieInstant.State.COMPLETED.toString().equals(record.get(ACTION_STATE).toString()));
+setInstants(instants);

Review comment:
   Oops, my fault, ignore this comment, the `readCommits` is updated in 
method `readCommit`.




-- 
This is an automated message from the 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-unsubscr...@hudi.apache.org

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




[GitHub] [hudi] melin commented on issue #4857: [SUPPORT]Failed to create a table

2022-02-21 Thread GitBox


melin commented on issue #4857:
URL: https://github.com/apache/hudi/issues/4857#issuecomment-1047373708


   > @melin would you please show your spark version and the code snippet to 
reproduce?
   
   Spark 3.2
   ```
   create table bigdata.test_hudi_dt (
   id int comment '',
   name string comment '',
   price double comment '',
   ds string comment ''
   ) USING hudi
   PARTITIONED BY (ds)
   OPTIONS (primaryKey = 'id', type = 'cow', preCombineField='ds');
   ```


-- 
This is an automated message from the 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-unsubscr...@hudi.apache.org

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




[jira] [Created] (HUDI-3472) Compilation failure for release-0.10.1

2022-02-21 Thread Leo zhang (Jira)
Leo zhang created HUDI-3472:
---

 Summary: Compilation failure for release-0.10.1
 Key: HUDI-3472
 URL: https://issues.apache.org/jira/browse/HUDI-3472
 Project: Apache Hudi
  Issue Type: Bug
  Components: dependencies
Affects Versions: 0.10.1
Reporter: Leo zhang


When I tried to compile then release-0.10.1, I met a compilation failure as 
below:

[ERROR] 
hudi/hudi-common/src/test/java/org/apache/hudi/common/testutils/FileCreateUtils.java:[22,37]
 package org.apache.directory.api.util does not exists.
[ERROR] 
hudi/hudi-common/src/test/java/org/apache/hudi/common/testutils/FileCreateUtils.java:[224,85]
 Can't find symbol
[ERROR]   Symbol:  variable Strings
[ERROR]   Position: Class org.apache.hudi.common.testutils.FileCreateUtils



It turns out this is caused by  lack of dependency for 

org.apache.directory.api:api-util, at module hudi-common.

 

Add a dependency to the pom can fix this problem.


org.apache.directory.api
api-util
1.0.0-M20




--
This message was sent by Atlassian Jira
(v8.20.1#820001)


[GitHub] [hudi] hudi-bot removed a comment on pull request #4862: [MINOR] Change MINI_BATCH_SIZE to 2048

2022-02-21 Thread GitBox


hudi-bot removed a comment on pull request #4862:
URL: https://github.com/apache/hudi/pull/4862#issuecomment-1047334883


   
   ## CI report:
   
   * 98b0b2816f33c4a2f5576e3581a11d1b9dd1ee26 Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6164)
 Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6169)
 Azure: 
[PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6184)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
This is an automated message from the 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-unsubscr...@hudi.apache.org

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




[GitHub] [hudi] hudi-bot commented on pull request #4862: [MINOR] Change MINI_BATCH_SIZE to 2048

2022-02-21 Thread GitBox


hudi-bot commented on pull request #4862:
URL: https://github.com/apache/hudi/pull/4862#issuecomment-1047369505


   
   ## CI report:
   
   * 98b0b2816f33c4a2f5576e3581a11d1b9dd1ee26 Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6164)
 Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6169)
 Azure: 
[SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6184)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
This is an automated message from the 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-unsubscr...@hudi.apache.org

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




[GitHub] [hudi] hudi-bot removed a comment on pull request #4848: [HUDI-3356][HUDI-3203] HoodieData for metadata index records, bloom and colstats init

2022-02-21 Thread GitBox


hudi-bot removed a comment on pull request #4848:
URL: https://github.com/apache/hudi/pull/4848#issuecomment-1047368221


   
   ## CI report:
   
   * 63aac434acd15223dc186635f963e97367e9 Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6124)
 
   * 97f253e3d9ef2c8caf05810d42e5f54e7598d4de UNKNOWN
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
This is an automated message from the 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-unsubscr...@hudi.apache.org

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




[GitHub] [hudi] hudi-bot commented on pull request #4848: [HUDI-3356][HUDI-3203] HoodieData for metadata index records, bloom and colstats init

2022-02-21 Thread GitBox


hudi-bot commented on pull request #4848:
URL: https://github.com/apache/hudi/pull/4848#issuecomment-1047369471


   
   ## CI report:
   
   * 63aac434acd15223dc186635f963e97367e9 Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6124)
 
   * 97f253e3d9ef2c8caf05810d42e5f54e7598d4de Azure: 
[PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6187)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
This is an automated message from the 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-unsubscr...@hudi.apache.org

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




  1   2   3   4   >