[jira] [Resolved] (HUDI-4769) Option read.streaming.skip_compaction skips delta commit

2022-10-01 Thread Danny Chen (Jira)


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

Danny Chen resolved HUDI-4769.
--

> Option read.streaming.skip_compaction skips delta commit
> 
>
> Key: HUDI-4769
> URL: https://issues.apache.org/jira/browse/HUDI-4769
> Project: Apache Hudi
>  Issue Type: Bug
>  Components: flink, flink-sql
>Reporter: nonggia.liang
>Assignee: Sagar Sumit
>Priority: Critical
>  Labels: pull-request-available
> Fix For: 0.13.0
>
>
> Option read.streaming.skip_compaction was introduced to avoid consuming 
> duplicate data from delta-commits and compactions in MOR table.
> But the option may cause delta-commits, here the case:
> Support we have a timeline (d for delta-commit, C for compaction/commit):
> d1 --> d2 --> C3 --> d4 --> d5 -->
> t1...t2..
> Let's say scans for streaming read happen at time t1 and t2, when d1 and d5 
> is the latest instant seperately. 
> When we scan at t2 with read.streaming.skip_compaction=true, we get a latest 
> merged fileslice with only log files containing d4+d5.  So d2 is skipped.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Commented] (HUDI-4769) Option read.streaming.skip_compaction skips delta commit

2022-10-01 Thread Danny Chen (Jira)


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

Danny Chen commented on HUDI-4769:
--

Fixed via master branch: f171496de244992958fd3fd22fbcd2a7dc62c7a2

> Option read.streaming.skip_compaction skips delta commit
> 
>
> Key: HUDI-4769
> URL: https://issues.apache.org/jira/browse/HUDI-4769
> Project: Apache Hudi
>  Issue Type: Bug
>  Components: flink, flink-sql
>Reporter: nonggia.liang
>Assignee: Sagar Sumit
>Priority: Critical
>  Labels: pull-request-available
> Fix For: 0.13.0
>
>
> Option read.streaming.skip_compaction was introduced to avoid consuming 
> duplicate data from delta-commits and compactions in MOR table.
> But the option may cause delta-commits, here the case:
> Support we have a timeline (d for delta-commit, C for compaction/commit):
> d1 --> d2 --> C3 --> d4 --> d5 -->
> t1...t2..
> Let's say scans for streaming read happen at time t1 and t2, when d1 and d5 
> is the latest instant seperately. 
> When we scan at t2 with read.streaming.skip_compaction=true, we get a latest 
> merged fileslice with only log files containing d4+d5.  So d2 is skipped.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[hudi] branch master updated: [HUDI-4769] Option read.streaming.skip_compaction skips delta commit (#6848)

2022-10-01 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 f171496de2 [HUDI-4769] Option read.streaming.skip_compaction skips 
delta commit (#6848)
f171496de2 is described below

commit f171496de244992958fd3fd22fbcd2a7dc62c7a2
Author: Danny Chan 
AuthorDate: Sun Oct 2 09:08:03 2022 +0800

[HUDI-4769] Option read.streaming.skip_compaction skips delta commit (#6848)
---
 .../table/view/AbstractTableFileSystemView.java|  50 ++
 .../apache/hudi/source/IncrementalInputSplits.java | 178 +
 .../hudi/source/StreamReadMonitoringFunction.java  |   5 +-
 .../apache/hudi/table/format/TestInputFormat.java  |  70 +++-
 4 files changed, 197 insertions(+), 106 deletions(-)

diff --git 
a/hudi-common/src/main/java/org/apache/hudi/common/table/view/AbstractTableFileSystemView.java
 
b/hudi-common/src/main/java/org/apache/hudi/common/table/view/AbstractTableFileSystemView.java
index ed4bfd7601..976217ae07 100644
--- 
a/hudi-common/src/main/java/org/apache/hudi/common/table/view/AbstractTableFileSystemView.java
+++ 
b/hudi-common/src/main/java/org/apache/hudi/common/table/view/AbstractTableFileSystemView.java
@@ -713,6 +713,33 @@ public abstract class AbstractTableFileSystemView 
implements SyncableFileSystemV
 }
   }
 
+  /**
+   * Stream all "merged" file-slices before on an instant time
+   * for a MERGE_ON_READ table with index that can index log files(which means 
it writes pure logs first).
+   *
+   * In streaming read scenario, in order for better reading efficiency, 
the user can choose to skip the
+   * base files that are produced by compaction. That is to say, we allow the 
users to consumer only from
+   * these partitioned log files, these log files keep the record sequence 
just like the normal message queue.
+   *
+   * NOTE: only local view is supported.
+   *
+   * @param partitionStr   Partition Path
+   * @param maxInstantTime Max Instant Time
+   */
+  public final Stream getAllLogsMergedFileSliceBeforeOrOn(String 
partitionStr, String maxInstantTime) {
+try {
+  readLock.lock();
+  String partition = formatPartitionKey(partitionStr);
+  ensurePartitionLoadedCorrectly(partition);
+  return fetchAllStoredFileGroups(partition)
+  .filter(fg -> !isFileGroupReplacedBeforeOrOn(fg.getFileGroupId(), 
maxInstantTime))
+  .map(fileGroup -> fetchAllLogsMergedFileSlice(fileGroup, 
maxInstantTime))
+  
.filter(Option::isPresent).map(Option::get).map(this::addBootstrapBaseFileIfPresent);
+} finally {
+  readLock.unlock();
+}
+  }
+
   @Override
   public final Stream getLatestFileSliceInRange(List 
commitsToReturn) {
 try {
@@ -1076,6 +1103,29 @@ public abstract class AbstractTableFileSystemView 
implements SyncableFileSystemV
 return fileSlice;
   }
 
+  /**
+   * Returns the file slice with all the file slice log files merged.
+   *
+   * @param fileGroup File Group for which the file slice belongs to
+   * @param maxInstantTime The max instant time
+   */
+  private Option fetchAllLogsMergedFileSlice(HoodieFileGroup 
fileGroup, String maxInstantTime) {
+List fileSlices = 
fileGroup.getAllFileSlicesBeforeOn(maxInstantTime).collect(Collectors.toList());
+if (fileSlices.size() == 0) {
+  return Option.empty();
+}
+if (fileSlices.size() == 1) {
+  return Option.of(fileSlices.get(0));
+}
+final FileSlice latestSlice = fileSlices.get(0);
+FileSlice merged = new FileSlice(latestSlice.getPartitionPath(), 
latestSlice.getBaseInstantTime(),
+latestSlice.getFileId());
+
+// add log files from the latest slice to the earliest
+fileSlices.forEach(slice -> 
slice.getLogFiles().forEach(merged::addLogFile));
+return Option.of(merged);
+  }
+
   /**
* Default implementation for fetching latest base-file.
*
diff --git 
a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/source/IncrementalInputSplits.java
 
b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/source/IncrementalInputSplits.java
index dc10970b05..09f7054cd7 100644
--- 
a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/source/IncrementalInputSplits.java
+++ 
b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/source/IncrementalInputSplits.java
@@ -20,6 +20,7 @@ package org.apache.hudi.source;
 
 import org.apache.hudi.common.fs.FSUtils;
 import org.apache.hudi.common.model.BaseFile;
+import org.apache.hudi.common.model.FileSlice;
 import org.apache.hudi.common.model.HoodieCommitMetadata;
 import org.apache.hudi.common.model.HoodieFileGroupId;
 import org.apache.hudi.common.model.HoodieLogFile;
@@ -220,7 +221,7 @@ public class IncrementalInputSplits implements Serializable 
{
 : instants.get(instants.size() - 1).getTimestamp();
 
 List inputSplits = 

[GitHub] [hudi] danny0405 merged pull request #6848: [HUDI-4769] Option read.streaming.skip_compaction skips delta commit

2022-10-01 Thread GitBox


danny0405 merged PR #6848:
URL: https://github.com/apache/hudi/pull/6848


-- 
This is an automated message from the 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 #6848: [HUDI-4769] Option read.streaming.skip_compaction skips delta commit

2022-10-01 Thread GitBox


hudi-bot commented on PR #6848:
URL: https://github.com/apache/hudi/pull/6848#issuecomment-1264507836

   
   ## CI report:
   
   * fcfc1baee8aa5394c4472135f6ceb0d280cf8c16 Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11955)
 
   * 856ec633afc90a499aa4de4f69ada1439a63f7a4 Azure: 
[PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11960)
 
   
   
   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 #6848: [HUDI-4769] Option read.streaming.skip_compaction skips delta commit

2022-10-01 Thread GitBox


hudi-bot commented on PR #6848:
URL: https://github.com/apache/hudi/pull/6848#issuecomment-1264507171

   
   ## CI report:
   
   * fcfc1baee8aa5394c4472135f6ceb0d280cf8c16 Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11955)
 
   * 856ec633afc90a499aa4de4f69ada1439a63f7a4 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 #5958: [HUDI-3900] [UBER] Support log compaction action for MOR tables

2022-10-01 Thread GitBox


hudi-bot commented on PR #5958:
URL: https://github.com/apache/hudi/pull/5958#issuecomment-1264506981

   
   ## CI report:
   
   * c7a7b3a42376116bbeaa09c1f0f07a4f80d788b6 Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11959)
 
   
   
   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 pull request #6848: [HUDI-4769] Option read.streaming.skip_compaction skips delta commit

2022-10-01 Thread GitBox


danny0405 commented on PR #6848:
URL: https://github.com/apache/hudi/pull/6848#issuecomment-1264499469

   The failed test `ITTestDataStreamWrite.testWriteCopyOnWriteWithClustering` 
is flaky, it should not affected by this pr, would merge it soon and fire 
another PR to fix the tests separately.


-- 
This is an automated message from the 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 #5958: [HUDI-3900] [UBER] Support log compaction action for MOR tables

2022-10-01 Thread GitBox


hudi-bot commented on PR #5958:
URL: https://github.com/apache/hudi/pull/5958#issuecomment-1264478333

   
   ## CI report:
   
   * 4590a5c3ccb91255a3abb0b9f478325cecfad59b Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11932)
 
   * c7a7b3a42376116bbeaa09c1f0f07a4f80d788b6 Azure: 
[PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11959)
 
   
   
   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 #5958: [HUDI-3900] [UBER] Support log compaction action for MOR tables

2022-10-01 Thread GitBox


hudi-bot commented on PR #5958:
URL: https://github.com/apache/hudi/pull/5958#issuecomment-1264477514

   
   ## CI report:
   
   * 4590a5c3ccb91255a3abb0b9f478325cecfad59b Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11932)
 
   * c7a7b3a42376116bbeaa09c1f0f07a4f80d788b6 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] nsivabalan commented on a diff in pull request #5958: [HUDI-3900] [UBER] Support log compaction action for MOR tables

2022-10-01 Thread GitBox


nsivabalan commented on code in PR #5958:
URL: https://github.com/apache/hudi/pull/5958#discussion_r985130392


##
hudi-common/src/main/java/org/apache/hudi/common/config/HoodieMetadataConfig.java:
##
@@ -234,6 +234,13 @@ public final class HoodieMetadataConfig extends 
HoodieConfig {
   + "metadata table which are never added before. This config 
determines how to handle "
   + "such spurious deletes");
 
+  public static final ConfigProperty USE_LOG_RECORD_READER_SCAN_V2 = 
ConfigProperty
+  .key(METADATA_PREFIX + ".log.record.reader.use.scanV2")
+  .defaultValue(false)
+  .sinceVersion("0.10.10")

Review Comment:
   0.13.0



##
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/RunCompactionActionExecutor.java:
##
@@ -50,29 +49,35 @@
 
   private final HoodieCompactor compactor;
   private final HoodieCompactionHandler compactionHandler;
+  private WriteOperationType operationType;
 
   public RunCompactionActionExecutor(HoodieEngineContext context,
  HoodieWriteConfig config,
  HoodieTable table,
  String instantTime,
  HoodieCompactor compactor,
- HoodieCompactionHandler 
compactionHandler) {
+ HoodieCompactionHandler compactionHandler,
+ WriteOperationType operationType) {
 super(context, config, table, instantTime);
 this.compactor = compactor;
 this.compactionHandler = compactionHandler;
+this.operationType = operationType;

Review Comment:
   can we assert that type can be either COMPACT or LOG_COMACT. bcoz, in other 
methods below, in else blocks we don't check for type. so it could be anything



##
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/CompactionExecutionStrategy.java:
##
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *  http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.table.action.compact;
+
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.model.CompactionOperation;
+import org.apache.hudi.common.model.HoodieBaseFile;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner;
+import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.common.table.timeline.HoodieTimeline;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.table.HoodieCompactionHandler;
+import org.apache.hudi.table.HoodieTable;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Iterator;
+import java.util.List;
+
+public class CompactionExecutionStrategy implements Serializable {
+
+  protected void transitionRequestedToInflight(HoodieTable table, String 
compactionInstantTime) {
+HoodieActiveTimeline timeline = table.getActiveTimeline();
+HoodieInstant instant = 
HoodieTimeline.getCompactionRequestedInstant(compactionInstantTime);
+// Mark instant as compaction inflight
+timeline.transitionCompactionRequestedToInflight(instant);
+table.getMetaClient().reloadActiveTimeline();
+  }
+
+  protected String instantTimeToUseForScanning(String compactionInstantTime, 
String maxInstantTime) {
+return maxInstantTime;
+  }
+
+  protected boolean shouldPreserveCommitMetadata() {
+return false;

Review Comment:
   this is not right. we should fetch this from CompactionConfig. 



##
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieAppendHandle.java:
##
@@ -379,11 +407,16 @@ public void doAppend() {
   flushToDiskIfRequired(record);
   writeToBuffer(record);
 }
-appendDataAndDeleteBlocks(header);
+appendDataAndDeleteBlocks(header, true);
 estimatedNumberOfBytesWritten += averageRecordSize * numberOfRecords;
   }
 
-  protected void appendDataAndDeleteBlocks(Map 
header) {
+  /**
+   * Appends data and 

[GitHub] [hudi] nsivabalan commented on a diff in pull request #5958: [HUDI-3900] [UBER] Support log compaction action for MOR tables

2022-10-01 Thread GitBox


nsivabalan commented on code in PR #5958:
URL: https://github.com/apache/hudi/pull/5958#discussion_r985130069


##
hudi-common/src/main/java/org/apache/hudi/common/table/log/AbstractHoodieLogRecordReader.java:
##
@@ -362,6 +381,228 @@ protected synchronized void scanInternal(Option 
keySpecOpt) {
 }
   }
 
+  private void scanInternalV2(Option keySpecOption, boolean 
skipProcessingBlocks) {

Review Comment:
   yes. we are good.



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

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 #6805: [HUDI-4949] optimize cdc read to avoid the problem of reusing buffer underlying the Row

2022-10-01 Thread GitBox


hudi-bot commented on PR #6805:
URL: https://github.com/apache/hudi/pull/6805#issuecomment-1264423605

   
   ## CI report:
   
   * 8b5baf6007d88f619ca2838c507c93d48e3c7f45 Azure: 
[SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11958)
 
   
   
   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] [Created] (HUDI-4965) automatically adapt COMMITS_ARCHIVAL_BATCH_SIZE

2022-10-01 Thread ZiyueGuan (Jira)
ZiyueGuan created HUDI-4965:
---

 Summary: automatically adapt COMMITS_ARCHIVAL_BATCH_SIZE
 Key: HUDI-4965
 URL: https://issues.apache.org/jira/browse/HUDI-4965
 Project: Apache Hudi
  Issue Type: Improvement
Reporter: ZiyueGuan


COMMITS_ARCHIVAL_BATCH_SIZE is used to determine how many records will write to 
archived timeline. However, if there are several huge instants which leads to a 
huge batch, it cannot be written as log block is overflow. So determine batch 
size by instants binary size.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[GitHub] [hudi] hudi-bot commented on pull request #6745: Fix comment in RFC46

2022-10-01 Thread GitBox


hudi-bot commented on PR #6745:
URL: https://github.com/apache/hudi/pull/6745#issuecomment-1264403083

   
   ## CI report:
   
   * a21ec4c91ba3d353082411e46239c9c81912cc1f Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11957)
 
   
   
   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 a diff in pull request #5416: [HUDI-3963] Use Lock-Free Message Queue Disruptor Improving Hoodie Writing Efficiency

2022-10-01 Thread GitBox


nsivabalan commented on code in PR #5416:
URL: https://github.com/apache/hudi/pull/5416#discussion_r985104403


##
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java:
##
@@ -129,6 +131,14 @@ public class HoodieWriteConfig extends HoodieConfig {
   .withDocumentation("Key generator class, that implements 
`org.apache.hudi.keygen.KeyGenerator` "
   + "extract a key out of incoming records.");
 
+  public static final ConfigProperty EXECUTOR_TYPE = ConfigProperty
+  .key("hoodie.write.executor.type")
+  .defaultValue("BOUNDED_IN_MEMORY_EXECUTOR")

Review Comment:
   can we introduce enum for this. also, we can remove "EXECUTOR" suffix from 
it. for eg, in case of Key gen, enum is named as KeyGeneratorType and value is 
just "SIMPLE". we don't call it as "SIMPOLE_KEY_GEN" as its repetitive. 
   



##
hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieDisruptorEvent.java:
##
@@ -0,0 +1,36 @@
+/*
+ * 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.common.util.queue;
+
+public class HoodieDisruptorEvent {

Review Comment:
   java docs



##
hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/execution/benchmark/BoundInMemoryExecutorBenchmark.scala:
##
@@ -0,0 +1,133 @@
+/*
+ * 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.spark.sql.execution.benchmark
+
+import org.apache.hadoop.fs.Path
+import org.apache.hudi.HoodieSparkUtils
+import org.apache.spark.SparkConf
+import org.apache.spark.hudi.benchmark.{HoodieBenchmark, HoodieBenchmarkBase}
+import org.apache.spark.sql.hudi.HoodieSparkSessionExtension
+import org.apache.spark.sql.types._
+import org.apache.spark.sql.{DataFrame, RowFactory, SaveMode, SparkSession}
+
+import scala.util.Random
+
+object BoundInMemoryExecutorBenchmark extends HoodieBenchmarkBase {
+
+  protected val spark: SparkSession = getSparkSession
+
+  val recordNumber = 100
+
+  def getSparkSession: SparkSession = SparkSession.builder()
+.master("local[*]")
+.appName(this.getClass.getCanonicalName)
+.withExtensions(new HoodieSparkSessionExtension)
+.config("spark.serializer", "org.apache.spark.serializer.KryoSerializer")
+.config("spark.sql.session.timeZone", "CTT")
+.config(sparkConf())
+.getOrCreate()
+
+  def sparkConf(): SparkConf = {
+val sparkConf = new SparkConf()
+if (HoodieSparkUtils.gteqSpark3_2) {
+  sparkConf.set("spark.sql.catalog.spark_catalog",
+"org.apache.spark.sql.hudi.catalog.HoodieCatalog")
+}
+sparkConf
+  }
+
+  private def createDataFrame(number: Int): DataFrame = {
+val schema = new StructType()
+  .add("c1", IntegerType)
+  .add("c2", StringType)
+
+val rdd = spark.sparkContext.parallelize(0 to number, 2).map { item =>
+  val c1 = Integer.valueOf(item)
+  val c2 = s"abc"
+  RowFactory.create(c1, c2)
+}
+spark.createDataFrame(rdd, schema)
+  }
+
+  /**
+   * Intel(R) Xeon(R) Platinum 8259CL CPU @ 2.50GHz
+   * COW Ingestion:Best Time(ms)   Avg Time(ms)   
Stdev(ms)Rate(M/s)   Per Row(ns)   Relative
+   * 

+   * BoundInMemory Executor 5557   5607
  70  0.25556.9   1.0X

[GitHub] [hudi] melin commented on pull request #4309: [HUDI-3016][RFC-43] Proposal to implement Table Management Service

2022-10-01 Thread GitBox


melin commented on PR #4309:
URL: https://github.com/apache/hudi/pull/4309#issuecomment-1264388044

   Support multi catalog?
   


-- 
This is an automated message from the 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 #6805: [HUDI-4949] optimize cdc read to avoid the problem of reusing buffer underlying the Row

2022-10-01 Thread GitBox


hudi-bot commented on PR #6805:
URL: https://github.com/apache/hudi/pull/6805#issuecomment-1264387150

   
   ## CI report:
   
   * be8c12ec97231e1dd549cb3ece42a388e2976563 Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11925)
 
   * 8b5baf6007d88f619ca2838c507c93d48e3c7f45 Azure: 
[PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11958)
 
   
   
   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 #6805: [HUDI-4949] optimize cdc read to avoid the problem of reusing buffer underlying the Row

2022-10-01 Thread GitBox


hudi-bot commented on PR #6805:
URL: https://github.com/apache/hudi/pull/6805#issuecomment-1264386089

   
   ## CI report:
   
   * be8c12ec97231e1dd549cb3ece42a388e2976563 Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11925)
 
   * 8b5baf6007d88f619ca2838c507c93d48e3c7f45 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 #6848: [HUDI-4769] Option read.streaming.skip_compaction skips delta commit

2022-10-01 Thread GitBox


hudi-bot commented on PR #6848:
URL: https://github.com/apache/hudi/pull/6848#issuecomment-1264384029

   
   ## CI report:
   
   * fcfc1baee8aa5394c4472135f6ceb0d280cf8c16 Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11955)
 
   
   
   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 #6745: Fix comment in RFC46

2022-10-01 Thread GitBox


hudi-bot commented on PR #6745:
URL: https://github.com/apache/hudi/pull/6745#issuecomment-1264365805

   
   ## CI report:
   
   * b7a71b7de263c5117e199df45cba0ead191159a3 Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11938)
 Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11952)
 
   * a21ec4c91ba3d353082411e46239c9c81912cc1f Azure: 
[PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11957)
 
   
   
   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 #6745: Fix comment in RFC46

2022-10-01 Thread GitBox


hudi-bot commented on PR #6745:
URL: https://github.com/apache/hudi/pull/6745#issuecomment-1264364720

   
   ## CI report:
   
   * b7a71b7de263c5117e199df45cba0ead191159a3 Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11938)
 Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11952)
 
   * a21ec4c91ba3d353082411e46239c9c81912cc1f 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] [Closed] (HUDI-4936) as.of.instant not recognized as hoodie config

2022-10-01 Thread Raymond Xu (Jira)


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

Raymond Xu closed HUDI-4936.

Resolution: Fixed

> as.of.instant not recognized as hoodie config
> -
>
> Key: HUDI-4936
> URL: https://issues.apache.org/jira/browse/HUDI-4936
> Project: Apache Hudi
>  Issue Type: Bug
>  Components: spark-sql
>Reporter: Raymond Xu
>Assignee: Raymond Xu
>Priority: Major
>  Labels: pull-request-available
> Fix For: 0.12.1
>
>
> original issue
> https://github.com/apache/hudi/issues/5586



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Updated] (HUDI-1380) Async cleaning does not work with Timeline Server

2022-10-01 Thread Zhaojing Yu (Jira)


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

Zhaojing Yu updated HUDI-1380:
--
Fix Version/s: 0.13.0
   (was: 0.12.1)

> Async cleaning does not work with Timeline Server
> -
>
> Key: HUDI-1380
> URL: https://issues.apache.org/jira/browse/HUDI-1380
> Project: Apache Hudi
>  Issue Type: Bug
>  Components: Common Core, table-service, timeline-server
>Reporter: Nishith Agarwal
>Priority: Major
> Fix For: 0.13.0
>
>




--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Updated] (HUDI-1574) Trim existing unit tests to finish in much shorter amount of time

2022-10-01 Thread Zhaojing Yu (Jira)


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

Zhaojing Yu updated HUDI-1574:
--
Fix Version/s: 0.13.0
   (was: 0.12.1)

> Trim existing unit tests to finish in much shorter amount of time
> -
>
> Key: HUDI-1574
> URL: https://issues.apache.org/jira/browse/HUDI-1574
> Project: Apache Hudi
>  Issue Type: Epic
>  Components: Testing, tests-ci
>Affects Versions: 0.9.0
>Reporter: Vinoth Chandar
>Priority: Critical
> Fix For: 0.13.0
>
>
> spark-client-tests
> 278.165 s - in org.apache.hudi.table.TestHoodieMergeOnReadTable
> 201.628 s - in org.apache.hudi.metadata.TestHoodieBackedMetadata
> 185.716 s - in org.apache.hudi.client.TestHoodieClientOnCopyOnWriteStorage
> 158.361 s - in org.apache.hudi.index.TestHoodieIndex
> 156.196 s - in org.apache.hudi.table.TestCleaner
> 132.369 s - in 
> org.apache.hudi.table.action.commit.TestCopyOnWriteActionExecutor
> 93.307 s - in org.apache.hudi.table.action.compact.TestAsyncCompaction
> 67.301 s - in org.apache.hudi.table.upgrade.TestUpgradeDowngrade
> 45.794 s - in org.apache.hudi.client.TestHoodieReadClient
> 38.615 s - in org.apache.hudi.index.bloom.TestHoodieBloomIndex
> 31.181 s - in org.apache.hudi.client.TestTableSchemaEvolution
> 20.072 s - in org.apache.hudi.table.action.compact.TestInlineCompaction
> grep " Time elapsed" hudi-client/hudi-spark-client/target/surefire-reports/* 
> | awk -F',' ' { print $5 } ' | awk -F':' ' { print $2 } ' | sort -nr | less
> hudi-utilities
> 209.936 s - in org.apache.hudi.utilities.functional.TestHoodieDeltaStreamer
> 204.653 s - in 
> org.apache.hudi.utilities.functional.TestHoodieMultiTableDeltaStreamer
> 34.116 s - in org.apache.hudi.utilities.sources.TestKafkaSource
> 29.865 s - in org.apache.hudi.utilities.sources.TestParquetDFSSource
> 26.189 s - in 
> org.apache.hudi.utilities.sources.helpers.TestDatePartitionPathSelector
> Other Tests
> 42.595 s - in org.apache.hudi.common.functional.TestHoodieLogFormat
> 38.918 s - in org.apache.hudi.common.bootstrap.TestBootstrapIndex
> 22.046 s - in 
> org.apache.hudi.common.functional.TestHoodieLogFormatAppendFailure



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Updated] (HUDI-2465) Fix merge, update for spark sql dml support to test suite infra

2022-10-01 Thread Zhaojing Yu (Jira)


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

Zhaojing Yu updated HUDI-2465:
--
Fix Version/s: 0.13.0
   (was: 0.12.1)

> Fix merge, update for spark sql dml support to test suite infra
> ---
>
> Key: HUDI-2465
> URL: https://issues.apache.org/jira/browse/HUDI-2465
> Project: Apache Hudi
>  Issue Type: Test
>  Components: spark, Testing, tests-ci
>Reporter: sivabalan narayanan
>Assignee: sivabalan narayanan
>Priority: Major
> Fix For: 0.13.0
>
>
> We have some dep issue wrt merge and update. we need to fix it. 



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Updated] (HUDI-1885) Support Delete/Update Non-Pk Table

2022-10-01 Thread Zhaojing Yu (Jira)


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

Zhaojing Yu updated HUDI-1885:
--
Fix Version/s: 0.13.0
   (was: 0.12.1)

> Support Delete/Update Non-Pk Table
> --
>
> Key: HUDI-1885
> URL: https://issues.apache.org/jira/browse/HUDI-1885
> Project: Apache Hudi
>  Issue Type: New Feature
>  Components: spark, spark-sql
>Reporter: pengzhiwei
>Assignee: Yann Byron
>Priority: Critical
> Fix For: 0.13.0
>
>
> Allow to delete/update a non-pk table.
> {code:java}
> create table h0 (
>   id int,
>   name string,
>   price double
> ) using hudi;
> delete from h0 where id = 10;
> update h0 set price = 10 where id = 12;
> {code}



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Updated] (HUDI-2528) Flaky test: MERGE_ON_READ testTableOperationsWithRestore

2022-10-01 Thread Zhaojing Yu (Jira)


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

Zhaojing Yu updated HUDI-2528:
--
Fix Version/s: 0.13.0
   (was: 0.12.1)

> Flaky test: MERGE_ON_READ testTableOperationsWithRestore
> 
>
> Key: HUDI-2528
> URL: https://issues.apache.org/jira/browse/HUDI-2528
> Project: Apache Hudi
>  Issue Type: Bug
>  Components: Testing, tests-ci
>Reporter: Raymond Xu
>Priority: Blocker
> Fix For: 0.13.0
>
>
>  
> {code:java}
>  [ERROR] Failures:[ERROR] There files should have been rolled-back when 
> rolling back commit 002 but are still remaining. Files: 
> [file:/tmp/junit6464799159313857398/2016/03/15/9d59f0f1-9cfa-41a4-b247-6bf002ad6cc7-0_0-592-8761_001.parquet,
>  
> file:/tmp/junit6464799159313857398/2016/03/15/9d59f0f1-9cfa-41a4-b247-6bf002ad6cc7-0_0-585-8754_001.parquet]
>  ==> expected: <0> but was: <2>[ERROR] Errors:[ERROR] No Compaction 
> request available at 007 to run compaction {code}
>  
> Probably the same cause as HUDI-2108
>  



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Updated] (HUDI-2555) Fix flaky FlinkCompaction integration test

2022-10-01 Thread Zhaojing Yu (Jira)


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

Zhaojing Yu updated HUDI-2555:
--
Fix Version/s: 0.13.0
   (was: 0.12.1)

> Fix flaky FlinkCompaction integration test
> --
>
> Key: HUDI-2555
> URL: https://issues.apache.org/jira/browse/HUDI-2555
> Project: Apache Hudi
>  Issue Type: Improvement
>Reporter: sivabalan narayanan
>Assignee: Danny Chen
>Priority: Major
> Fix For: 0.13.0
>
>
> Recently CI was broken and had to revert some suspicious tests. 
> [https://github.com/apache/hudi/pull/3793]
> We need to fix it and re-enable them back. 
> [ITTestHoodieFlinkCompactor.java|https://github.com/apache/hudi/pull/3793/files#diff-f15b4ec18c40c9494e62ae73aa4b79beeafd1a5fa185b6ec6a7044fa6ed9e1fd]
>  
> testHoodieFlinkCompactor



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Updated] (HUDI-2768) Enable async timeline server by default

2022-10-01 Thread Zhaojing Yu (Jira)


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

Zhaojing Yu updated HUDI-2768:
--
Fix Version/s: 0.13.0
   (was: 0.12.1)

> Enable async timeline server by default
> ---
>
> Key: HUDI-2768
> URL: https://issues.apache.org/jira/browse/HUDI-2768
> Project: Apache Hudi
>  Issue Type: Improvement
>  Components: timeline-server, writer-core
>Reporter: sivabalan narayanan
>Assignee: Ethan Guo
>Priority: Critical
>  Labels: hudi-on-call, pull-request-available
> Fix For: 0.13.0
>
>
> Enable async timeline server by default.
>  
> [https://github.com/apache/hudi/pull/3949]
>  
>  



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Updated] (HUDI-2928) Evaluate rebasing Hudi's default compression from Gzip to Zstd

2022-10-01 Thread Zhaojing Yu (Jira)


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

Zhaojing Yu updated HUDI-2928:
--
Fix Version/s: 0.13.0
   (was: 0.12.1)

> Evaluate rebasing Hudi's default compression from Gzip to Zstd
> --
>
> Key: HUDI-2928
> URL: https://issues.apache.org/jira/browse/HUDI-2928
> Project: Apache Hudi
>  Issue Type: Improvement
>  Components: performance, storage-management
>Reporter: Alexey Kudinkin
>Assignee: Alexey Kudinkin
>Priority: Critical
>  Labels: pull-request-available
> Fix For: 0.13.0
>
> Attachments: Screen Shot 2021-12-03 at 12.36.13 PM.png, Screen Shot 
> 2021-12-06 at 11.49.05 AM.png, image-2021-12-03-13-13-02-892.png
>
>
> Currently, having Gzip as a default we prioritize Compression/Storage cost at 
> the expense of
>  * Compute (on the {+}write-path{+}): about *30%* of Compute burned during 
> bulk-insert in local benchmarks on Amazon Reviews dataset is Gzip (see below) 
>  * Compute (on the {+}read-path{+}), as well as queries Latencies: queries 
> scanning large datasets are likely to be compression-/CPU-bound (Gzip t/put 
> is *3-4x* less than Snappy, Zstd, 
> [EX|https://stackoverflow.com/a/56410326/3520840])
> P.S Spark switched its default compression algorithm to Snappy [a while 
> ago|https://github.com/apache/spark/pull/12256].
>  
> *EDIT*
> We should actually evaluate putting in 
> [zstd|https://engineering.fb.com/2016/08/31/core-data/smaller-and-faster-data-compression-with-zstandard/]
>  instead of Snappy. It has compression ratios comparable to Gzip, while 
> bringing in much better performance:
> !image-2021-12-03-13-13-02-892.png!
> [https://engineering.fb.com/2016/08/31/core-data/smaller-and-faster-data-compression-with-zstandard/]
>  
>  
>  



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Resolved] (HUDI-3054) Fix flaky TestHoodieClientMultiWriter. testHoodieClientBasicMultiWriter

2022-10-01 Thread Zhaojing Yu (Jira)


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

Zhaojing Yu resolved HUDI-3054.
---

> Fix flaky TestHoodieClientMultiWriter. testHoodieClientBasicMultiWriter
> ---
>
> Key: HUDI-3054
> URL: https://issues.apache.org/jira/browse/HUDI-3054
> Project: Apache Hudi
>  Issue Type: Test
>  Components: Testing, tests-ci
>Reporter: sivabalan narayanan
>Assignee: sivabalan narayanan
>Priority: Blocker
>  Labels: pull-request-available
> Fix For: 0.12.1
>
>
> Ref: 
> [https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_apis/build/builds/4428/logs/21]
>  
> {code:java}
> 2021-12-17T11:39:57.1645757Z [INFO] Running 
> org.apache.hudi.client.TestHoodieClientMultiWriter
> 2021-12-17T11:39:57.3453991Z 339506 [main] WARN  
> org.apache.hudi.metadata.HoodieBackedTableMetadata  - Metadata table was not 
> found at path /tmp/junit8865530218583640556/dataset/.hoodie/metadata
> 2021-12-17T11:39:57.3984328Z 339559 [dispatcher-event-loop-5] WARN  
> org.apache.spark.scheduler.TaskSetManager  - Stage 0 contains a task of very 
> large size (101 KB). The maximum recommended task size is 100 KB.
> 2021-12-17T11:39:57.5278608Z 339689 [dispatcher-event-loop-2] WARN  
> org.apache.spark.scheduler.TaskSetManager  - Stage 1 contains a task of very 
> large size (101 KB). The maximum recommended task size is 100 KB.
> 2021-12-17T11:39:57.9783107Z 340139 [main] WARN  
> org.apache.hudi.metadata.HoodieBackedTableMetadata  - Metadata table was not 
> found at path /tmp/junit8865530218583640556/dataset/.hoodie/metadata
> 2021-12-17T11:39:57.9927490Z 340154 [main] WARN  
> org.apache.hudi.metadata.HoodieBackedTableMetadata  - Metadata table was not 
> found at path /tmp/junit8865530218583640556/dataset/.hoodie/metadata
> 2021-12-17T11:40:10.1428665Z 352304 [main] WARN  
> org.apache.hudi.testutils.HoodieClientTestHarness  - Closing file-system 
> instance used in previous test-run
> 2021-12-17T11:40:10.9930023Z 353149 [main] WARN  
> org.apache.hudi.testutils.HoodieClientTestHarness  - Closing file-system 
> instance used in previous test-run
> 2021-12-17T11:40:11.4294603Z 353590 [main] WARN  
> org.apache.hudi.metadata.HoodieBackedTableMetadata  - Metadata table was not 
> found at path /tmp/junit3262960667280061850/dataset/.hoodie/metadata
> 2021-12-17T11:40:11.4763085Z 353637 [dispatcher-event-loop-5] WARN  
> org.apache.spark.scheduler.TaskSetManager  - Stage 0 contains a task of very 
> large size (101 KB). The maximum recommended task size is 100 KB.
> 2021-12-17T11:40:11.6014876Z 353762 [dispatcher-event-loop-2] WARN  
> org.apache.spark.scheduler.TaskSetManager  - Stage 1 contains a task of very 
> large size (101 KB). The maximum recommended task size is 100 KB.
> 2021-12-17T11:40:12.0892513Z 354250 [main] WARN  
> org.apache.hudi.metadata.HoodieBackedTableMetadata  - Metadata table was not 
> found at path /tmp/junit3262960667280061850/dataset/.hoodie/metadata
> 2021-12-17T11:40:12.1061317Z 354267 [main] WARN  
> org.apache.hudi.metadata.HoodieBackedTableMetadata  - Metadata table was not 
> found at path /tmp/junit3262960667280061850/dataset/.hoodie/metadata
> 2021-12-17T11:40:23.1499732Z 365311 [main] WARN  
> org.apache.hudi.testutils.HoodieClientTestHarness  - Closing file-system 
> instance used in previous test-run
> 2021-12-17T11:40:24.1626167Z 366323 [main] WARN  
> org.apache.hudi.metadata.HoodieBackedTableMetadata  - Metadata table was not 
> found at path /tmp/junit294667857867877904/dataset/.hoodie/metadata
> 2021-12-17T11:40:24.1945944Z 366355 [dispatcher-event-loop-5] WARN  
> org.apache.spark.scheduler.TaskSetManager  - Stage 0 contains a task of very 
> large size (101 KB). The maximum recommended task size is 100 KB.
> 2021-12-17T11:40:24.3084730Z 366469 [dispatcher-event-loop-2] WARN  
> org.apache.spark.scheduler.TaskSetManager  - Stage 1 contains a task of very 
> large size (101 KB). The maximum recommended task size is 100 KB.
> 2021-12-17T11:40:24.7350862Z 366896 [main] WARN  
> org.apache.hudi.metadata.HoodieBackedTableMetadata  - Metadata table was not 
> found at path /tmp/junit294667857867877904/dataset/.hoodie/metadata
> 2021-12-17T11:40:24.7482727Z 366909 [main] WARN  
> org.apache.hudi.metadata.HoodieBackedTableMetadata  - Metadata table was not 
> found at path /tmp/junit294667857867877904/dataset/.hoodie/metadata
> 2021-12-17T11:40:43.1530857Z 385314 [main] WARN  
> org.apache.hudi.testutils.HoodieClientTestHarness  - Closing file-system 
> instance used in previous test-run
> 2021-12-17T11:40:44.0641298Z 386225 [main] WARN  
> org.apache.hudi.testutils.HoodieClientTestHarness  - Closing file-system 
> instance used in previous test-run
> 2021-12-17T11:40:44.5065216Z 386667 [main] 

[jira] [Updated] (HUDI-2860) Make timeline server work with concurrent/async table service

2022-10-01 Thread Zhaojing Yu (Jira)


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

Zhaojing Yu updated HUDI-2860:
--
Fix Version/s: 0.13.0
   (was: 0.12.1)

> Make timeline server work with concurrent/async table service
> -
>
> Key: HUDI-2860
> URL: https://issues.apache.org/jira/browse/HUDI-2860
> Project: Apache Hudi
>  Issue Type: Improvement
>  Components: table-service, writer-core
>Reporter: sivabalan narayanan
>Priority: Critical
> Fix For: 0.13.0
>
>
> Make timeline server work with multiple concurrent writers. 
> As of now, if an executor is lagging wrt timeline server (timeline server 
> refreshes its state for every call if timeline has moved), we throw an 
> exception and executor falls back to secondary which will list the file 
> system. 
>  
> Related ticket: https://issues.apache.org/jira/browse/HUDI-2761
>  
> We want to revisit this code and see how can we make timeline server work 
> with multi-writer scenario. 
>  
> Few points to consider:
> 1. Executors should try to call getLatestBaseFilesOnOrBefore() instead of 
> getLatestBaseFiles(). Not calls has to be fixed. the ones doing conflict 
> resolutions, might have to get the latest snapshot always. 
> 2. Fix async services to use separate write client in deltastreamer flow
> 3. Revist every call from executor and set "REFRESH" param on only when 
> matters.
> 4. Sharing embedded timeline server. 
> 5. Check for any holes. when C100 and C101 concurrently started and C101 
> finishes early, if C100 makes getLatestBaseFileOnOrBefore(), do we return 
> base files from C101? 
>  
>  



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Updated] (HUDI-3000) [UMBRELLA] Consistent Hashing Index

2022-10-01 Thread Zhaojing Yu (Jira)


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

Zhaojing Yu updated HUDI-3000:
--
Fix Version/s: 0.13.0
   (was: 0.12.1)

> [UMBRELLA] Consistent Hashing Index
> ---
>
> Key: HUDI-3000
> URL: https://issues.apache.org/jira/browse/HUDI-3000
> Project: Apache Hudi
>  Issue Type: Epic
>  Components: index
>Reporter: Yuwei Xiao
>Priority: Major
> Fix For: 0.13.0
>
>




--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Updated] (HUDI-2873) Support optimize data layout by sql and make the build more fast

2022-10-01 Thread Zhaojing Yu (Jira)


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

Zhaojing Yu updated HUDI-2873:
--
Fix Version/s: 0.13.0
   (was: 0.12.1)

> Support optimize data layout by sql and make the build more fast
> 
>
> Key: HUDI-2873
> URL: https://issues.apache.org/jira/browse/HUDI-2873
> Project: Apache Hudi
>  Issue Type: New Feature
>  Components: performance, spark
>Reporter: tao meng
>Assignee: shibei
>Priority: Critical
> Fix For: 0.13.0
>
>




--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Updated] (HUDI-3057) Instants should be generated strictly under locks

2022-10-01 Thread Zhaojing Yu (Jira)


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

Zhaojing Yu updated HUDI-3057:
--
Fix Version/s: 0.13.0
   (was: 0.12.1)

> Instants should be generated strictly under locks
> -
>
> Key: HUDI-3057
> URL: https://issues.apache.org/jira/browse/HUDI-3057
> Project: Apache Hudi
>  Issue Type: Bug
>  Components: multi-writer, writer-core
>Reporter: Alexey Kudinkin
>Assignee: sivabalan narayanan
>Priority: Major
>  Labels: sev:high
> Fix For: 0.13.0
>
> Attachments: logs.txt
>
>
> While looking into the flakiness of the tests outlined here:
> https://issues.apache.org/jira/browse/HUDI-3043
>  
> I've stumbled upon following failure where one of the writers tries to 
> complete the Commit but it couldn't b/c such file does already exist:
> {code:java}
> java.util.concurrent.ExecutionException: java.lang.RuntimeException: 
> org.apache.hudi.exception.HoodieIOException: Failed to create file 
> /var/folders/kb/cnff55vj041g2nnlzs5ylqk0gn/T/junit5142536255031969586/testtable_MERGE_ON_READ/.hoodie/20211217150157632.commit
>     at java.util.concurrent.FutureTask.report(FutureTask.java:122)
>     at java.util.concurrent.FutureTask.get(FutureTask.java:192)
>     at 
> org.apache.hudi.utilities.functional.TestHoodieDeltaStreamerWithMultiWriter.runJobsInParallel(TestHoodieDeltaStreamerWithMultiWriter.java:336)
>     at 
> org.apache.hudi.utilities.functional.TestHoodieDeltaStreamerWithMultiWriter.testUpsertsContinuousModeWithMultipleWriters(TestHoodieDeltaStreamerWithMultiWriter.java:150)
>     at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>     at 
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
>     at 
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>     at java.lang.reflect.Method.invoke(Method.java:498)
>     at 
> org.junit.platform.commons.util.ReflectionUtils.invokeMethod(ReflectionUtils.java:688)
>     at 
> org.junit.jupiter.engine.execution.MethodInvocation.proceed(MethodInvocation.java:60)
>     at 
> org.junit.jupiter.engine.execution.InvocationInterceptorChain$ValidatingInvocation.proceed(InvocationInterceptorChain.java:131)
>     at 
> org.junit.jupiter.engine.extension.TimeoutExtension.intercept(TimeoutExtension.java:149)
>     at 
> org.junit.jupiter.engine.extension.TimeoutExtension.interceptTestableMethod(TimeoutExtension.java:140)
>     at 
> org.junit.jupiter.engine.extension.TimeoutExtension.interceptTestTemplateMethod(TimeoutExtension.java:92)
>     at 
> org.junit.jupiter.engine.execution.ExecutableInvoker$ReflectiveInterceptorCall.lambda$ofVoidMethod$0(ExecutableInvoker.java:115)
>     at 
> org.junit.jupiter.engine.execution.ExecutableInvoker.lambda$invoke$0(ExecutableInvoker.java:105)
>     at 
> org.junit.jupiter.engine.execution.InvocationInterceptorChain$InterceptedInvocation.proceed(InvocationInterceptorChain.java:106)
>     at 
> org.junit.jupiter.engine.execution.InvocationInterceptorChain.proceed(InvocationInterceptorChain.java:64)
>     at 
> org.junit.jupiter.engine.execution.InvocationInterceptorChain.chainAndInvoke(InvocationInterceptorChain.java:45)
>     at 
> org.junit.jupiter.engine.execution.InvocationInterceptorChain.invoke(InvocationInterceptorChain.java:37)
>     at 
> org.junit.jupiter.engine.execution.ExecutableInvoker.invoke(ExecutableInvoker.java:104)
>     at 
> org.junit.jupiter.engine.execution.ExecutableInvoker.invoke(ExecutableInvoker.java:98)
>     at 
> org.junit.jupiter.engine.descriptor.TestMethodTestDescriptor.lambda$invokeTestMethod$6(TestMethodTestDescriptor.java:212)
>     at 
> org.junit.platform.engine.support.hierarchical.ThrowableCollector.execute(ThrowableCollector.java:73)
>     at 
> org.junit.jupiter.engine.descriptor.TestMethodTestDescriptor.invokeTestMethod(TestMethodTestDescriptor.java:208)
>     at 
> org.junit.jupiter.engine.descriptor.TestMethodTestDescriptor.execute(TestMethodTestDescriptor.java:137)
>     at 
> org.junit.jupiter.engine.descriptor.TestMethodTestDescriptor.execute(TestMethodTestDescriptor.java:71)
>     at 
> org.junit.platform.engine.support.hierarchical.NodeTestTask.lambda$executeRecursively$5(NodeTestTask.java:139)
>     at 
> org.junit.platform.engine.support.hierarchical.ThrowableCollector.execute(ThrowableCollector.java:73)
>     at 
> org.junit.platform.engine.support.hierarchical.NodeTestTask.lambda$executeRecursively$7(NodeTestTask.java:129)
>     at 
> org.junit.platform.engine.support.hierarchical.Node.around(Node.java:137)
>     at 
> org.junit.platform.engine.support.hierarchical.NodeTestTask.lambda$executeRecursively$8(NodeTestTask.java:127)
>     at 
> 

[jira] [Updated] (HUDI-3054) Fix flaky TestHoodieClientMultiWriter. testHoodieClientBasicMultiWriter

2022-10-01 Thread Zhaojing Yu (Jira)


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

Zhaojing Yu updated HUDI-3054:
--
Fix Version/s: 0.12.1

> Fix flaky TestHoodieClientMultiWriter. testHoodieClientBasicMultiWriter
> ---
>
> Key: HUDI-3054
> URL: https://issues.apache.org/jira/browse/HUDI-3054
> Project: Apache Hudi
>  Issue Type: Test
>  Components: Testing, tests-ci
>Reporter: sivabalan narayanan
>Assignee: sivabalan narayanan
>Priority: Blocker
>  Labels: pull-request-available
> Fix For: 0.12.1
>
>
> Ref: 
> [https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_apis/build/builds/4428/logs/21]
>  
> {code:java}
> 2021-12-17T11:39:57.1645757Z [INFO] Running 
> org.apache.hudi.client.TestHoodieClientMultiWriter
> 2021-12-17T11:39:57.3453991Z 339506 [main] WARN  
> org.apache.hudi.metadata.HoodieBackedTableMetadata  - Metadata table was not 
> found at path /tmp/junit8865530218583640556/dataset/.hoodie/metadata
> 2021-12-17T11:39:57.3984328Z 339559 [dispatcher-event-loop-5] WARN  
> org.apache.spark.scheduler.TaskSetManager  - Stage 0 contains a task of very 
> large size (101 KB). The maximum recommended task size is 100 KB.
> 2021-12-17T11:39:57.5278608Z 339689 [dispatcher-event-loop-2] WARN  
> org.apache.spark.scheduler.TaskSetManager  - Stage 1 contains a task of very 
> large size (101 KB). The maximum recommended task size is 100 KB.
> 2021-12-17T11:39:57.9783107Z 340139 [main] WARN  
> org.apache.hudi.metadata.HoodieBackedTableMetadata  - Metadata table was not 
> found at path /tmp/junit8865530218583640556/dataset/.hoodie/metadata
> 2021-12-17T11:39:57.9927490Z 340154 [main] WARN  
> org.apache.hudi.metadata.HoodieBackedTableMetadata  - Metadata table was not 
> found at path /tmp/junit8865530218583640556/dataset/.hoodie/metadata
> 2021-12-17T11:40:10.1428665Z 352304 [main] WARN  
> org.apache.hudi.testutils.HoodieClientTestHarness  - Closing file-system 
> instance used in previous test-run
> 2021-12-17T11:40:10.9930023Z 353149 [main] WARN  
> org.apache.hudi.testutils.HoodieClientTestHarness  - Closing file-system 
> instance used in previous test-run
> 2021-12-17T11:40:11.4294603Z 353590 [main] WARN  
> org.apache.hudi.metadata.HoodieBackedTableMetadata  - Metadata table was not 
> found at path /tmp/junit3262960667280061850/dataset/.hoodie/metadata
> 2021-12-17T11:40:11.4763085Z 353637 [dispatcher-event-loop-5] WARN  
> org.apache.spark.scheduler.TaskSetManager  - Stage 0 contains a task of very 
> large size (101 KB). The maximum recommended task size is 100 KB.
> 2021-12-17T11:40:11.6014876Z 353762 [dispatcher-event-loop-2] WARN  
> org.apache.spark.scheduler.TaskSetManager  - Stage 1 contains a task of very 
> large size (101 KB). The maximum recommended task size is 100 KB.
> 2021-12-17T11:40:12.0892513Z 354250 [main] WARN  
> org.apache.hudi.metadata.HoodieBackedTableMetadata  - Metadata table was not 
> found at path /tmp/junit3262960667280061850/dataset/.hoodie/metadata
> 2021-12-17T11:40:12.1061317Z 354267 [main] WARN  
> org.apache.hudi.metadata.HoodieBackedTableMetadata  - Metadata table was not 
> found at path /tmp/junit3262960667280061850/dataset/.hoodie/metadata
> 2021-12-17T11:40:23.1499732Z 365311 [main] WARN  
> org.apache.hudi.testutils.HoodieClientTestHarness  - Closing file-system 
> instance used in previous test-run
> 2021-12-17T11:40:24.1626167Z 366323 [main] WARN  
> org.apache.hudi.metadata.HoodieBackedTableMetadata  - Metadata table was not 
> found at path /tmp/junit294667857867877904/dataset/.hoodie/metadata
> 2021-12-17T11:40:24.1945944Z 366355 [dispatcher-event-loop-5] WARN  
> org.apache.spark.scheduler.TaskSetManager  - Stage 0 contains a task of very 
> large size (101 KB). The maximum recommended task size is 100 KB.
> 2021-12-17T11:40:24.3084730Z 366469 [dispatcher-event-loop-2] WARN  
> org.apache.spark.scheduler.TaskSetManager  - Stage 1 contains a task of very 
> large size (101 KB). The maximum recommended task size is 100 KB.
> 2021-12-17T11:40:24.7350862Z 366896 [main] WARN  
> org.apache.hudi.metadata.HoodieBackedTableMetadata  - Metadata table was not 
> found at path /tmp/junit294667857867877904/dataset/.hoodie/metadata
> 2021-12-17T11:40:24.7482727Z 366909 [main] WARN  
> org.apache.hudi.metadata.HoodieBackedTableMetadata  - Metadata table was not 
> found at path /tmp/junit294667857867877904/dataset/.hoodie/metadata
> 2021-12-17T11:40:43.1530857Z 385314 [main] WARN  
> org.apache.hudi.testutils.HoodieClientTestHarness  - Closing file-system 
> instance used in previous test-run
> 2021-12-17T11:40:44.0641298Z 386225 [main] WARN  
> org.apache.hudi.testutils.HoodieClientTestHarness  - Closing file-system 
> instance used in previous test-run
> 

[jira] [Resolved] (HUDI-3204) Allow original partition column value to be retrieved when using TimestampBasedKeyGen

2022-10-01 Thread Zhaojing Yu (Jira)


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

Zhaojing Yu resolved HUDI-3204.
---

> Allow original partition column value to be retrieved when using 
> TimestampBasedKeyGen
> -
>
> Key: HUDI-3204
> URL: https://issues.apache.org/jira/browse/HUDI-3204
> Project: Apache Hudi
>  Issue Type: Bug
>  Components: spark
>Reporter: Yann Byron
>Assignee: Alexey Kudinkin
>Priority: Blocker
>  Labels: hudi-on-call, pull-request-available, sev:critical
> Fix For: 0.12.1
>
>   Original Estimate: 3h
>  Time Spent: 1h
>  Remaining Estimate: 1h
>
> {color:#172b4d}Currently, b/c Spark by default omits partition values from 
> the data files (instead encoding them into partition paths for partitioned 
> tables), using `TimestampBasedKeyGenerator` w/ original timestamp 
> based-column makes it impossible to retrieve the original value (reading from 
> Spark) even though it's persisted in the data file as well.{color}
>  
> {code:java}
> import org.apache.hudi.DataSourceWriteOptions
> import org.apache.hudi.config.HoodieWriteConfig
> import org.apache.hudi.keygen.constant.KeyGeneratorOptions._
> import org.apache.hudi.hive.MultiPartKeysValueExtractor
> val df = Seq((1, "z3", 30, "v1", "2018-09-23"), (2, "z3", 35, "v1", 
> "2018-09-24")).toDF("id", "name", "age", "ts", "data_date")
> // mor
> df.write.format("hudi").
> option(HoodieWriteConfig.TABLE_NAME, "issue_4417_mor").
> option("hoodie.datasource.write.table.type", "MERGE_ON_READ").
> option("hoodie.datasource.write.recordkey.field", "id").
> option("hoodie.datasource.write.partitionpath.field", "data_date").
> option("hoodie.datasource.write.precombine.field", "ts").
> option("hoodie.datasource.write.keygenerator.class", 
> "org.apache.hudi.keygen.TimestampBasedKeyGenerator").
> option("hoodie.deltastreamer.keygen.timebased.timestamp.type", "DATE_STRING").
> option("hoodie.deltastreamer.keygen.timebased.output.dateformat", 
> "/MM/dd").
> option("hoodie.deltastreamer.keygen.timebased.timezone", "GMT+8:00").
> option("hoodie.deltastreamer.keygen.timebased.input.dateformat", 
> "-MM-dd").
> mode(org.apache.spark.sql.SaveMode.Append).
> save("file:///tmp/hudi/issue_4417_mor")
> +---++--+--++---++---+---+--+
> |_hoodie_commit_time|_hoodie_commit_seqno|_hoodie_record_key|_hoodie_partition_path|
>    _hoodie_file_name| id|name|age| ts| data_date|
> +---++--+--++---++---+---+--+
> |  20220110172709324|20220110172709324...|                 2|            
> 2018/09/24|703e56d3-badb-40b...|  2|  z3| 35| v1|2018-09-24|
> |  20220110172709324|20220110172709324...|                 1|            
> 2018/09/23|58fde2b3-db0e-464...|  1|  z3| 30| v1|2018-09-23|
> +---++--+--++---++---+---+--+
> // can not query any data
> spark.read.format("hudi").load("file:///tmp/hudi/issue_4417_mor").where("data_date
>  = '2018-09-24'")
> // still can not query any data
> spark.read.format("hudi").load("file:///tmp/hudi/issue_4417_mor").where("data_date
>  = '2018/09/24'").show 
> // cow
> df.write.format("hudi").
> option(HoodieWriteConfig.TABLE_NAME, "issue_4417_cow").
> option("hoodie.datasource.write.table.type", "COPY_ON_WRITE").
> option("hoodie.datasource.write.recordkey.field", "id").
> option("hoodie.datasource.write.partitionpath.field", "data_date").
> option("hoodie.datasource.write.precombine.field", "ts").
> option("hoodie.datasource.write.keygenerator.class", 
> "org.apache.hudi.keygen.TimestampBasedKeyGenerator").
> option("hoodie.deltastreamer.keygen.timebased.timestamp.type", "DATE_STRING").
> option("hoodie.deltastreamer.keygen.timebased.output.dateformat", 
> "/MM/dd").
> option("hoodie.deltastreamer.keygen.timebased.timezone", "GMT+8:00").
> option("hoodie.deltastreamer.keygen.timebased.input.dateformat", 
> "-MM-dd").
> mode(org.apache.spark.sql.SaveMode.Append).
> save("file:///tmp/hudi/issue_4417_cow") 
> +---++--+--++---++---+---+--+
>  
> |_hoodie_commit_time|_hoodie_commit_seqno|_hoodie_record_key|_hoodie_partition_path|
>    _hoodie_file_name| id|name|age| ts| data_date| 
> +---++--+--++---++---+---+--+
>  |  20220110172721896|20220110172721896...|                 2|            
> 2018/09/24|81cc7819-a0d1-4e6...|  2|  z3| 35| 

[jira] [Updated] (HUDI-3210) [UMBRELLA] A new Presto connector for Hudi

2022-10-01 Thread Zhaojing Yu (Jira)


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

Zhaojing Yu updated HUDI-3210:
--
Fix Version/s: 0.13.0
   (was: 0.12.1)

> [UMBRELLA] A new Presto connector for Hudi
> --
>
> Key: HUDI-3210
> URL: https://issues.apache.org/jira/browse/HUDI-3210
> Project: Apache Hudi
>  Issue Type: Epic
>  Components: trino-presto
>Reporter: Todd Gao
>Assignee: Sagar Sumit
>Priority: Major
> Fix For: 0.13.0, 1.0.0
>
>
> This JIRA tracks all the tasks related to building a new Hudi connector in 
> Presto.
> h4.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Updated] (HUDI-3067) "Table already exists" error with multiple writers and dynamodb

2022-10-01 Thread Zhaojing Yu (Jira)


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

Zhaojing Yu updated HUDI-3067:
--
Fix Version/s: 0.13.0
   (was: 0.12.1)

> "Table already exists" error with multiple writers and dynamodb
> ---
>
> Key: HUDI-3067
> URL: https://issues.apache.org/jira/browse/HUDI-3067
> Project: Apache Hudi
>  Issue Type: Bug
>Reporter: Nikita Sheremet
>Assignee: Wenning Ding
>Priority: Critical
>  Labels: hudi-on-call
> Fix For: 0.13.0
>
>
> How reproduce:
>  # Set up multiple writing 
> [https://hudi.apache.org/docs/concurrency_control/] for dynamodb (do not 
> forget to set _hoodie.write.lock.dynamodb.region_ and 
> {_}hoodie.write.lock.dynamodb.billing_mode{_}). Do not create anty dynamodb 
> table.
>  # Run multiple writers to the table
> (Tested on aws EMR, so multiple writers is EMR steps)
> Expected result - all steps completed.
> Actual result: some steps failed with exception 
> {code:java}
> Caused by: com.amazonaws.services.dynamodbv2.model.ResourceInUseException: 
> Table already exists: truedata_detections (Service: AmazonDynamoDBv2; Status 
> Code: 400; Error Code: ResourceInUseException; Request ID:; Proxy: null)
>   at 
> com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleErrorResponse(AmazonHttpClient.java:1819)
>   at 
> com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleServiceErrorResponse(AmazonHttpClient.java:1403)
>   at 
> com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeOneRequest(AmazonHttpClient.java:1372)
>   at 
> com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeHelper(AmazonHttpClient.java:1145)
>   at 
> com.amazonaws.http.AmazonHttpClient$RequestExecutor.doExecute(AmazonHttpClient.java:802)
>   at 
> com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeWithTimer(AmazonHttpClient.java:770)
>   at 
> com.amazonaws.http.AmazonHttpClient$RequestExecutor.execute(AmazonHttpClient.java:744)
>   at 
> com.amazonaws.http.AmazonHttpClient$RequestExecutor.access$500(AmazonHttpClient.java:704)
>   at 
> com.amazonaws.http.AmazonHttpClient$RequestExecutionBuilderImpl.execute(AmazonHttpClient.java:686)
>   at 
> com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:550)
>   at 
> com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:530)
>   at 
> com.amazonaws.services.dynamodbv2.AmazonDynamoDBClient.doInvoke(AmazonDynamoDBClient.java:6214)
>   at 
> com.amazonaws.services.dynamodbv2.AmazonDynamoDBClient.invoke(AmazonDynamoDBClient.java:6181)
>   at 
> com.amazonaws.services.dynamodbv2.AmazonDynamoDBClient.executeCreateTable(AmazonDynamoDBClient.java:1160)
>   at 
> com.amazonaws.services.dynamodbv2.AmazonDynamoDBClient.createTable(AmazonDynamoDBClient.java:1124)
>   at 
> org.apache.hudi.aws.transaction.lock.DynamoDBBasedLockProvider.createLockTableInDynamoDB(DynamoDBBasedLockProvider.java:188)
>   at 
> org.apache.hudi.aws.transaction.lock.DynamoDBBasedLockProvider.(DynamoDBBasedLockProvider.java:99)
>   at 
> org.apache.hudi.aws.transaction.lock.DynamoDBBasedLockProvider.(DynamoDBBasedLockProvider.java:77)
>   ... 54 more
> 21/12/19 13:42:06 INFO Yar {code}
> This happens because all steps tried to create table at the same time.
>  
> Suggested solution:
> A catch statment for _Table already exists_ exception should be added into 
> dynamodb table creation code. May be with delay and additional check that 
> table is present.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Updated] (HUDI-3054) Fix flaky TestHoodieClientMultiWriter. testHoodieClientBasicMultiWriter

2022-10-01 Thread Zhaojing Yu (Jira)


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

Zhaojing Yu updated HUDI-3054:
--
Fix Version/s: (was: 0.12.1)

> Fix flaky TestHoodieClientMultiWriter. testHoodieClientBasicMultiWriter
> ---
>
> Key: HUDI-3054
> URL: https://issues.apache.org/jira/browse/HUDI-3054
> Project: Apache Hudi
>  Issue Type: Test
>  Components: Testing, tests-ci
>Reporter: sivabalan narayanan
>Assignee: sivabalan narayanan
>Priority: Blocker
>  Labels: pull-request-available
>
> Ref: 
> [https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_apis/build/builds/4428/logs/21]
>  
> {code:java}
> 2021-12-17T11:39:57.1645757Z [INFO] Running 
> org.apache.hudi.client.TestHoodieClientMultiWriter
> 2021-12-17T11:39:57.3453991Z 339506 [main] WARN  
> org.apache.hudi.metadata.HoodieBackedTableMetadata  - Metadata table was not 
> found at path /tmp/junit8865530218583640556/dataset/.hoodie/metadata
> 2021-12-17T11:39:57.3984328Z 339559 [dispatcher-event-loop-5] WARN  
> org.apache.spark.scheduler.TaskSetManager  - Stage 0 contains a task of very 
> large size (101 KB). The maximum recommended task size is 100 KB.
> 2021-12-17T11:39:57.5278608Z 339689 [dispatcher-event-loop-2] WARN  
> org.apache.spark.scheduler.TaskSetManager  - Stage 1 contains a task of very 
> large size (101 KB). The maximum recommended task size is 100 KB.
> 2021-12-17T11:39:57.9783107Z 340139 [main] WARN  
> org.apache.hudi.metadata.HoodieBackedTableMetadata  - Metadata table was not 
> found at path /tmp/junit8865530218583640556/dataset/.hoodie/metadata
> 2021-12-17T11:39:57.9927490Z 340154 [main] WARN  
> org.apache.hudi.metadata.HoodieBackedTableMetadata  - Metadata table was not 
> found at path /tmp/junit8865530218583640556/dataset/.hoodie/metadata
> 2021-12-17T11:40:10.1428665Z 352304 [main] WARN  
> org.apache.hudi.testutils.HoodieClientTestHarness  - Closing file-system 
> instance used in previous test-run
> 2021-12-17T11:40:10.9930023Z 353149 [main] WARN  
> org.apache.hudi.testutils.HoodieClientTestHarness  - Closing file-system 
> instance used in previous test-run
> 2021-12-17T11:40:11.4294603Z 353590 [main] WARN  
> org.apache.hudi.metadata.HoodieBackedTableMetadata  - Metadata table was not 
> found at path /tmp/junit3262960667280061850/dataset/.hoodie/metadata
> 2021-12-17T11:40:11.4763085Z 353637 [dispatcher-event-loop-5] WARN  
> org.apache.spark.scheduler.TaskSetManager  - Stage 0 contains a task of very 
> large size (101 KB). The maximum recommended task size is 100 KB.
> 2021-12-17T11:40:11.6014876Z 353762 [dispatcher-event-loop-2] WARN  
> org.apache.spark.scheduler.TaskSetManager  - Stage 1 contains a task of very 
> large size (101 KB). The maximum recommended task size is 100 KB.
> 2021-12-17T11:40:12.0892513Z 354250 [main] WARN  
> org.apache.hudi.metadata.HoodieBackedTableMetadata  - Metadata table was not 
> found at path /tmp/junit3262960667280061850/dataset/.hoodie/metadata
> 2021-12-17T11:40:12.1061317Z 354267 [main] WARN  
> org.apache.hudi.metadata.HoodieBackedTableMetadata  - Metadata table was not 
> found at path /tmp/junit3262960667280061850/dataset/.hoodie/metadata
> 2021-12-17T11:40:23.1499732Z 365311 [main] WARN  
> org.apache.hudi.testutils.HoodieClientTestHarness  - Closing file-system 
> instance used in previous test-run
> 2021-12-17T11:40:24.1626167Z 366323 [main] WARN  
> org.apache.hudi.metadata.HoodieBackedTableMetadata  - Metadata table was not 
> found at path /tmp/junit294667857867877904/dataset/.hoodie/metadata
> 2021-12-17T11:40:24.1945944Z 366355 [dispatcher-event-loop-5] WARN  
> org.apache.spark.scheduler.TaskSetManager  - Stage 0 contains a task of very 
> large size (101 KB). The maximum recommended task size is 100 KB.
> 2021-12-17T11:40:24.3084730Z 366469 [dispatcher-event-loop-2] WARN  
> org.apache.spark.scheduler.TaskSetManager  - Stage 1 contains a task of very 
> large size (101 KB). The maximum recommended task size is 100 KB.
> 2021-12-17T11:40:24.7350862Z 366896 [main] WARN  
> org.apache.hudi.metadata.HoodieBackedTableMetadata  - Metadata table was not 
> found at path /tmp/junit294667857867877904/dataset/.hoodie/metadata
> 2021-12-17T11:40:24.7482727Z 366909 [main] WARN  
> org.apache.hudi.metadata.HoodieBackedTableMetadata  - Metadata table was not 
> found at path /tmp/junit294667857867877904/dataset/.hoodie/metadata
> 2021-12-17T11:40:43.1530857Z 385314 [main] WARN  
> org.apache.hudi.testutils.HoodieClientTestHarness  - Closing file-system 
> instance used in previous test-run
> 2021-12-17T11:40:44.0641298Z 386225 [main] WARN  
> org.apache.hudi.testutils.HoodieClientTestHarness  - Closing file-system 
> instance used in previous test-run
> 2021-12-17T11:40:44.5065216Z 386667 [main] 

[jira] [Updated] (HUDI-3055) Make sure that Compression Codec configuration is respected across the board

2022-10-01 Thread Zhaojing Yu (Jira)


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

Zhaojing Yu updated HUDI-3055:
--
Fix Version/s: 0.13.0
   (was: 0.12.1)

> Make sure that Compression Codec configuration is respected across the board
> 
>
> Key: HUDI-3055
> URL: https://issues.apache.org/jira/browse/HUDI-3055
> Project: Apache Hudi
>  Issue Type: Bug
>  Components: storage-management
>Reporter: Alexey Kudinkin
>Assignee: Alexey Kudinkin
>Priority: Major
>  Labels: new-to-hudi
> Fix For: 0.13.0
>
>
> Currently there are quite a few places where we assume GZip as the 
> compression codec which is incorrect, given that this is configurable and 
> users might actually prefer to use different compression codec.
> Examples:
> [HoodieParquetDataBlock|https://github.com/apache/hudi/pull/4333/files#diff-798a773c6eef4011aef2da2b2fb71c25f753500548167b610021336ef6f14807]



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Updated] (HUDI-3214) Optimize auto partition in spark

2022-10-01 Thread Zhaojing Yu (Jira)


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

Zhaojing Yu updated HUDI-3214:
--
Fix Version/s: 0.13.0
   (was: 0.12.1)

> Optimize auto partition in spark
> 
>
> Key: HUDI-3214
> URL: https://issues.apache.org/jira/browse/HUDI-3214
> Project: Apache Hudi
>  Issue Type: Improvement
>  Components: spark, writer-core
>Reporter: Yann Byron
>Assignee: Yann Byron
>Priority: Critical
>  Labels: pull-request-available
> Fix For: 0.13.0
>
>   Original Estimate: 2h
>  Remaining Estimate: 2h
>
> recently, if partition's value has the format like 
> "pt1=/pt2=/pt3=" which split by slash, Hudi will partition 
> automatically. The directory of this table will have multi partition 
> structure.
> I think it's unpredictable. So create this umbrella task to optimize auto 
> partition in order to make the behavior more reasonable.
> Also, in hudi 0.8, schama will hold `pt1`, `pt2`, `pt3`, but not in 0.9+.
> There are a few of sub tasks:
>  * add a flag to control whether enable auto-partition, to make the default 
> behavior reasonable..
>  * achieve a new key generator designed specifically for this scenario.
>  * solve the bug about the different schema when enable 
> *hoodie.file.index.enable* or not in this case.
>  
> Test Codes: 
> {code:java}
> import org.apache.hudi.QuickstartUtils._
> import scala.collection.JavaConversions._
> import org.apache.spark.sql.SaveMode._
> import org.apache.hudi.DataSourceReadOptions._
> import org.apache.hudi.DataSourceWriteOptions._
> import org.apache.hudi.config.HoodieWriteConfig._
> val tableName = "hudi_trips_cow"
> val basePath = "file:///tmp/hudi_trips_cow"
> val dataGen = new DataGenerator
> val inserts = convertToStringList(dataGen.generateInserts(10))
> val df = spark.read.json(spark.sparkContext.parallelize(inserts, 2))
> val newDf = df.withColumn("partitionpath", regexp_replace($"partitionpath", 
> "(.*)(\\/){1}(.*)(\\/){1}", "continent=$1$2country=$3$4city="))
> newDf.write.format("hudi").
> options(getQuickstartWriteConfigs).
> option(PRECOMBINE_FIELD_OPT_KEY, "ts").
> option(RECORDKEY_FIELD_OPT_KEY, "uuid").
> option(PARTITIONPATH_FIELD_OPT_KEY, "partitionpath").
> option(TABLE_NAME, tableName).
> mode(Overwrite).
> save(basePath) {code}



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Updated] (HUDI-3249) Performance Improvements

2022-10-01 Thread Zhaojing Yu (Jira)


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

Zhaojing Yu updated HUDI-3249:
--
Fix Version/s: (was: 0.12.1)

> Performance Improvements
> 
>
> Key: HUDI-3249
> URL: https://issues.apache.org/jira/browse/HUDI-3249
> Project: Apache Hudi
>  Issue Type: Epic
>  Components: writer-core
>Reporter: Ethan Guo
>Assignee: Ethan Guo
>Priority: Critical
> Fix For: 0.13.0
>
>




--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Updated] (HUDI-3300) Timeline server FSViewManager should avoid point lookup for metadata file partition

2022-10-01 Thread Zhaojing Yu (Jira)


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

Zhaojing Yu updated HUDI-3300:
--
Fix Version/s: 0.13.0

> Timeline server FSViewManager should avoid point lookup for metadata file 
> partition
> ---
>
> Key: HUDI-3300
> URL: https://issues.apache.org/jira/browse/HUDI-3300
> Project: Apache Hudi
>  Issue Type: Bug
>  Components: metadata, timeline-server
>Reporter: Manoj Govindassamy
>Assignee: Yue Zhang
>Priority: Major
> Fix For: 0.13.0
>
>
> When inline reading is enabled, that is 
> hoodie.metadata.enable.full.scan.log.files = false, 
> MetadataMergedLogRecordReader doesn't cache the file listings records via the 
> ExternalSpillableMap. So, every file listing will lead to re-reading of 
> metadata files partition log and base files. Since files partition is less in 
> size, even when inline reading is enabled, the TimelineServer should 
> construct the FSViewManager with inline reading disabled for metadata files 
> partition. 



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Updated] (HUDI-3300) Timeline server FSViewManager should avoid point lookup for metadata file partition

2022-10-01 Thread Zhaojing Yu (Jira)


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

Zhaojing Yu updated HUDI-3300:
--
Fix Version/s: (was: 0.12.1)

> Timeline server FSViewManager should avoid point lookup for metadata file 
> partition
> ---
>
> Key: HUDI-3300
> URL: https://issues.apache.org/jira/browse/HUDI-3300
> Project: Apache Hudi
>  Issue Type: Bug
>  Components: metadata, timeline-server
>Reporter: Manoj Govindassamy
>Assignee: Yue Zhang
>Priority: Major
>
> When inline reading is enabled, that is 
> hoodie.metadata.enable.full.scan.log.files = false, 
> MetadataMergedLogRecordReader doesn't cache the file listings records via the 
> ExternalSpillableMap. So, every file listing will lead to re-reading of 
> metadata files partition log and base files. Since files partition is less in 
> size, even when inline reading is enabled, the TimelineServer should 
> construct the FSViewManager with inline reading disabled for metadata files 
> partition. 



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Updated] (HUDI-3248) Improve Hudi table services

2022-10-01 Thread Zhaojing Yu (Jira)


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

Zhaojing Yu updated HUDI-3248:
--
Fix Version/s: 0.13.0
   (was: 0.12.1)

> Improve Hudi table services
> ---
>
> Key: HUDI-3248
> URL: https://issues.apache.org/jira/browse/HUDI-3248
> Project: Apache Hudi
>  Issue Type: Epic
>  Components: archiving, table-service, writer-core
>Reporter: Ethan Guo
>Assignee: Raymond Xu
>Priority: Blocker
> Fix For: 0.13.0
>
>




--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Updated] (HUDI-3301) MergedLogRecordReader inline reading should be stateless and thread safe

2022-10-01 Thread Zhaojing Yu (Jira)


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

Zhaojing Yu updated HUDI-3301:
--
Fix Version/s: 0.13.0
   (was: 0.12.1)

> MergedLogRecordReader inline reading should be stateless and thread safe
> 
>
> Key: HUDI-3301
> URL: https://issues.apache.org/jira/browse/HUDI-3301
> Project: Apache Hudi
>  Issue Type: Bug
>  Components: metadata
>Reporter: Manoj Govindassamy
>Assignee: Yue Zhang
>Priority: Major
> Fix For: 0.13.0
>
>
> Metadata table inline reading (enable.full.scan.log.files = false) today 
> alters instance member fields and not thread safe.
>  
> When the inline reading is enabled, HoodieMetadataMergedLogRecordReader 
> doesn't do full read of log and base files and doesn't fill in the 
> ExternalSpillableMap records cache. Each getRecordsByKeys() thereby will 
> re-read the log and base files by design. But the issue here is this reading 
> alters the instance members and the filled in records are relevant only for 
> that request. Any concurrent getRecordsByKeys() is also modifying the member 
> variable leading to NPE.
>  
> To avoid this, a temporary fix of making getRecordsByKeys() a synchronized 
> method has been pushed to master. But this fix doesn't solve all usecases. We 
> need to make the whole class stateless and thread safe for inline reading.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Updated] (HUDI-3342) MOR Delta Block Rollbacks not applied if Lazy Block reading is disabled

2022-10-01 Thread Zhaojing Yu (Jira)


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

Zhaojing Yu updated HUDI-3342:
--
Fix Version/s: 0.13.0
   (was: 0.12.1)

> MOR Delta Block Rollbacks not applied if Lazy Block reading is disabled
> ---
>
> Key: HUDI-3342
> URL: https://issues.apache.org/jira/browse/HUDI-3342
> Project: Apache Hudi
>  Issue Type: Bug
>  Components: metadata
>Reporter: Alexey Kudinkin
>Assignee: Raymond Xu
>Priority: Critical
> Fix For: 0.13.0
>
>
> While working on HUDI-3322, i've spotted following contraption:
> When we are rolling back Delta Commits, we add corresponding 
> {{ROLLBACK_PREVIOUS_BLOCK}} Command Block at the back of the "queue". When we 
> restore, we issue a sequence of Rollbacks, which means that stack if such 
> Rollback Blocks could be of size > 1.
> However, when reading that MOR table if the reader does not specify 
> `readBlocksLazily=true`, we'd be merging Blocks eagerly (when instants 
> increment) therefore essentially rendering such Rollback Blocks useless since 
> they can't "unmerge" previously merged records, resurrecting the data that 
> was supposed to be rolled back.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Updated] (HUDI-3517) Unicode in partition path causes it to be resolved wrongly

2022-10-01 Thread Zhaojing Yu (Jira)


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

Zhaojing Yu updated HUDI-3517:
--
Fix Version/s: 0.13.0
   (was: 0.12.1)

> Unicode in partition path causes it to be resolved wrongly
> --
>
> Key: HUDI-3517
> URL: https://issues.apache.org/jira/browse/HUDI-3517
> Project: Apache Hudi
>  Issue Type: Improvement
>  Components: spark-sql, writer-core
>Affects Versions: 0.10.1
>Reporter: Ji Qi
>Assignee: sivabalan narayanan
>Priority: Critical
>  Labels: hudi-on-call
> Fix For: 0.13.0
>
>   Original Estimate: 2h
>  Remaining Estimate: 2h
>
> When there is unicode in the partition path, the upsert fails.
> h3. To reproduce
>  # Create this dataframe in spark-shell (note the dotted I)
> {code:none}
> scala> res0.show(truncate=false)
> +---+---+
> |_c0|_c1|
> +---+---+
> |1  |İ  |
> +---+---+
> {code}
>  # Write it to hudi (this write will create the hudi table and succeed)
> {code:none}
>  res0.write.format("hudi").option("hoodie.table.name", 
> "unicode_test").option("hoodie.datasource.write.precombine.field", 
> "_c0").option("hoodie.datasource.write.recordkey.field", 
> "_c0").option("hoodie.datasource.write.partitionpath.field", 
> "_c1").mode("append").save("file:///Users/ji.qi/Desktop/unicode_test")
> {code}
>  # Try to write {{res0}} again (this upsert will fail at index lookup stage)
> Environment
>  * Hudi version: 0.10.1
>  * Spark version: 3.1.2
> h3. Stacktrace
> {code:none}
> 22/02/25 18:23:14 INFO RemoteHoodieTableFileSystemView: Sending request : 
> (http://192.168.1.148:54043/v1/hoodie/view/datafile/latest/partition?partition=%C4%B0=file%3A%2FUsers%2Fji.qi%2FDesktop%2Funicode_test=31517a5e-af56-4fbc-9aa6-1ef1729bb89d-0=20220225182311228=65c5a6a5c6836dc4f7805550e81ca034b30ad85c38794f9f8ce68a9e914aab83)
> 22/02/25 18:23:14 ERROR Executor: Exception in task 0.0 in stage 5.0 (TID 403)
> org.apache.hudi.exception.HoodieIOException: Failed to read footer for 
> parquet 
> file:/Users/ji.qi/Desktop/unicode_test/Ä°/31517a5e-af56-4fbc-9aa6-1ef1729bb89d-0_0-30-2006_20220225181656520.parquet
>   at 
> org.apache.hudi.common.util.ParquetUtils.readMetadata(ParquetUtils.java:185)
>   at 
> org.apache.hudi.common.util.ParquetUtils.readFooter(ParquetUtils.java:201)
>   at 
> org.apache.hudi.common.util.BaseFileUtils.readMinMaxRecordKeys(BaseFileUtils.java:109)
>   at 
> org.apache.hudi.io.storage.HoodieParquetReader.readMinMaxRecordKeys(HoodieParquetReader.java:49)
>   at 
> org.apache.hudi.io.HoodieRangeInfoHandle.getMinMaxKeys(HoodieRangeInfoHandle.java:39)
>   at 
> org.apache.hudi.index.bloom.HoodieBloomIndex.lambda$loadInvolvedFiles$4cbadf07$1(HoodieBloomIndex.java:149)
>   at 
> org.apache.spark.api.java.JavaPairRDD$.$anonfun$toScalaFunction$1(JavaPairRDD.scala:1070)
>   at scala.collection.Iterator$$anon$10.next(Iterator.scala:459)
>   at scala.collection.Iterator.foreach(Iterator.scala:941)
>   at scala.collection.Iterator.foreach$(Iterator.scala:941)
>   at scala.collection.AbstractIterator.foreach(Iterator.scala:1429)
>   at scala.collection.generic.Growable.$plus$plus$eq(Growable.scala:62)
>   at scala.collection.generic.Growable.$plus$plus$eq$(Growable.scala:53)
>   at 
> scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:105)
>   at 
> scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:49)
>   at scala.collection.TraversableOnce.to(TraversableOnce.scala:315)
>   at scala.collection.TraversableOnce.to$(TraversableOnce.scala:313)
>   at scala.collection.AbstractIterator.to(Iterator.scala:1429)
>   at scala.collection.TraversableOnce.toBuffer(TraversableOnce.scala:307)
>   at scala.collection.TraversableOnce.toBuffer$(TraversableOnce.scala:307)
>   at scala.collection.AbstractIterator.toBuffer(Iterator.scala:1429)
>   at scala.collection.TraversableOnce.toArray(TraversableOnce.scala:294)
>   at scala.collection.TraversableOnce.toArray$(TraversableOnce.scala:288)
>   at scala.collection.AbstractIterator.toArray(Iterator.scala:1429)
>   at org.apache.spark.rdd.RDD.$anonfun$collect$2(RDD.scala:1030)
>   at 
> org.apache.spark.SparkContext.$anonfun$runJob$5(SparkContext.scala:2236)
>   at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
>   at org.apache.spark.scheduler.Task.run(Task.scala:131)
>   at 
> org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:497)
>   at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1439)
>   at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:500)
>   at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
> 

[jira] [Updated] (HUDI-3495) Reading keys in parallel from HoodieMetadataMergedLogRecordReader may lead to empty results even if key exists

2022-10-01 Thread Zhaojing Yu (Jira)


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

Zhaojing Yu updated HUDI-3495:
--
Fix Version/s: 0.13.0
   (was: 0.12.1)

> Reading keys in parallel from HoodieMetadataMergedLogRecordReader may lead to 
> empty results even if key exists
> --
>
> Key: HUDI-3495
> URL: https://issues.apache.org/jira/browse/HUDI-3495
> Project: Apache Hudi
>  Issue Type: Bug
>  Components: metadata
>Reporter: Prashant Wason
>Assignee: Yue Zhang
>Priority: Blocker
> Fix For: 0.13.0
>
>
> [HoodieMetadataMergedLogRecordReader|https://github.com/apache/hudi/blob/master/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataMergedLogRecordReader.java]
>  has two function which lookup keys:
> getRecordByKey(String key) - lookups the key in member variable map "records"
> getRecordsByKeys(List keys) - clears member variable map "records" 
> and scans the log files again.
> If the two functions are called in parallel, the getRecordByKey() may return 
> an empty key because the records was cleared in another thread calling 
> getRecordsByKeys()



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Updated] (HUDI-3407) Make sure Restore operation is Not Concurrent w/ Writes in Multi-Writer scenario

2022-10-01 Thread Zhaojing Yu (Jira)


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

Zhaojing Yu updated HUDI-3407:
--
Fix Version/s: 0.13.0
   (was: 0.12.1)

> Make sure Restore operation is Not Concurrent w/ Writes in Multi-Writer 
> scenario
> 
>
> Key: HUDI-3407
> URL: https://issues.apache.org/jira/browse/HUDI-3407
> Project: Apache Hudi
>  Issue Type: Bug
>  Components: multi-writer
>Reporter: Alexey Kudinkin
>Priority: Critical
> Fix For: 0.13.0
>
>
> Currently there's no guard-rail that would prevent Restore from running 
> concurrently with Writes in Multi-Writer scenario, which might lead to table 
> getting into inconsistent state.
>  
> One of the approaches could be letting Restore to acquire the Write lock for 
> the whole duration of its operation.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


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

2022-10-01 Thread Zhaojing Yu (Jira)


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

Zhaojing Yu updated HUDI-3467:
--
Fix Version/s: 0.13.0
   (was: 0.12.1)

> 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
>  Components: compaction, spark
>Reporter: Ethan Guo
>Assignee: sivabalan narayanan
>Priority: Critical
>  Labels: hudi-on-call
> Fix For: 0.13.0
>
>
> Related issue
> https://github.com/apache/hudi/issues/5046



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Updated] (HUDI-3519) Make sure every public Hudi Client Method invokes necessary prologue

2022-10-01 Thread Zhaojing Yu (Jira)


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

Zhaojing Yu updated HUDI-3519:
--
Fix Version/s: 0.13.0

> Make sure every public Hudi Client Method invokes necessary prologue
> 
>
> Key: HUDI-3519
> URL: https://issues.apache.org/jira/browse/HUDI-3519
> Project: Apache Hudi
>  Issue Type: Improvement
>  Components: code-quality, metadata
>Reporter: Alexey Kudinkin
>Priority: Major
> Fix For: 0.13.0
>
>
> Right now, only a handful of operations actually invoke the "prologue" method 
> doing, for ex
>  # Checks around whether the table needs to be upgraded
>  # Bootstraps MDT (if necessary)
> As well as some other minor book-keeping stuff. As part of 
> [https://github.com/apache/hudi/pull/4739,] i had to address that and 
> introduced universal method `initTable` that serves as such prologue.
> However, while i've injected it into most major public methods of the Hudi 
> Client's Base class, we need to carefully and holistically review all 
> remaining exposed *public* methods and make sure that all _public-facing_ 
> operations (insert, upsert, commit, delete, rollback, clean, etc) are 
> invoking prologue properly.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Updated] (HUDI-3519) Make sure every public Hudi Client Method invokes necessary prologue

2022-10-01 Thread Zhaojing Yu (Jira)


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

Zhaojing Yu updated HUDI-3519:
--
Fix Version/s: (was: 0.12.1)

> Make sure every public Hudi Client Method invokes necessary prologue
> 
>
> Key: HUDI-3519
> URL: https://issues.apache.org/jira/browse/HUDI-3519
> Project: Apache Hudi
>  Issue Type: Improvement
>  Components: code-quality, metadata
>Reporter: Alexey Kudinkin
>Priority: Major
>
> Right now, only a handful of operations actually invoke the "prologue" method 
> doing, for ex
>  # Checks around whether the table needs to be upgraded
>  # Bootstraps MDT (if necessary)
> As well as some other minor book-keeping stuff. As part of 
> [https://github.com/apache/hudi/pull/4739,] i had to address that and 
> introduced universal method `initTable` that serves as such prologue.
> However, while i've injected it into most major public methods of the Hudi 
> Client's Base class, we need to carefully and holistically review all 
> remaining exposed *public* methods and make sure that all _public-facing_ 
> operations (insert, upsert, commit, delete, rollback, clean, etc) are 
> invoking prologue properly.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Updated] (HUDI-3648) Failed to execute rollback due to HoodieIOException: Could not delete instant

2022-10-01 Thread Zhaojing Yu (Jira)


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

Zhaojing Yu updated HUDI-3648:
--
Fix Version/s: 0.13.0
   (was: 0.12.1)

> Failed to execute rollback due to HoodieIOException: Could not delete instant
> -
>
> Key: HUDI-3648
> URL: https://issues.apache.org/jira/browse/HUDI-3648
> Project: Apache Hudi
>  Issue Type: Bug
>  Components: deltastreamer
>Reporter: Ethan Guo
>Assignee: Sagar Sumit
>Priority: Critical
>  Labels: hudi-on-call
> Fix For: 0.13.0
>
>
> Deltastreamer continuous mode writing to COW table with async clustering and 
> cleaning.
> {code:java}
> org.apache.hudi.exception.HoodieRollbackException: Failed to rollback 
> file:/Users/ethan/Work/scripts/mt_rollout_testing/deploy_b_single_writer_async_services/b3_ds_cow_010mt_011mt_conf/test_table
>  commits 20220314165647208
>     at 
> org.apache.hudi.client.BaseHoodieWriteClient.rollback(BaseHoodieWriteClient.java:695)
>     at 
> org.apache.hudi.client.BaseHoodieWriteClient.rollbackFailedWrites(BaseHoodieWriteClient.java:1037)
>     at 
> org.apache.hudi.client.BaseHoodieWriteClient.tryUpgrade(BaseHoodieWriteClient.java:1404)
>     at 
> org.apache.hudi.client.BaseHoodieWriteClient.initTable(BaseHoodieWriteClient.java:1302)
>     at 
> org.apache.hudi.client.SparkRDDWriteClient.insert(SparkRDDWriteClient.java:174)
>     at 
> org.apache.hudi.utilities.deltastreamer.DeltaSync.writeToSink(DeltaSync.java:574)
>     at 
> org.apache.hudi.utilities.deltastreamer.DeltaSync.syncOnce(DeltaSync.java:329)
>     at 
> org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer$DeltaSyncService.lambda$startService$0(HoodieDeltaStreamer.java:656)
>     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)
> Caused by: org.apache.hudi.exception.HoodieIOException: Could not delete 
> instant [==>20220314165647208__commit__INFLIGHT]
>     at 
> org.apache.hudi.common.table.timeline.HoodieActiveTimeline.deleteInstantFile(HoodieActiveTimeline.java:250)
>     at 
> org.apache.hudi.common.table.timeline.HoodieActiveTimeline.deletePending(HoodieActiveTimeline.java:201)
>     at 
> org.apache.hudi.table.action.rollback.BaseRollbackActionExecutor.deleteInflightAndRequestedInstant(BaseRollbackActionExecutor.java:270)
>     at 
> org.apache.hudi.table.action.rollback.CopyOnWriteRollbackActionExecutor.executeRollback(CopyOnWriteRollbackActionExecutor.java:90)
>     at 
> org.apache.hudi.table.action.rollback.BaseRollbackActionExecutor.doRollbackAndGetStats(BaseRollbackActionExecutor.java:218)
>     at 
> org.apache.hudi.table.action.rollback.BaseRollbackActionExecutor.runRollback(BaseRollbackActionExecutor.java:115)
>     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.rollback(BaseHoodieWriteClient.java:680)
>     ... 11 more {code}



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Updated] (HUDI-3646) The Hudi update syntax should not modify the nullability attribute of a column

2022-10-01 Thread Zhaojing Yu (Jira)


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

Zhaojing Yu updated HUDI-3646:
--
Fix Version/s: 0.13.0
   (was: 0.12.1)

> The Hudi update syntax should not modify the nullability attribute of a column
> --
>
> Key: HUDI-3646
> URL: https://issues.apache.org/jira/browse/HUDI-3646
> Project: Apache Hudi
>  Issue Type: Bug
>  Components: spark-sql
>Affects Versions: 0.10.1
> Environment: spark3.1.2
>Reporter: Tao Meng
>Assignee: Alexey Kudinkin
>Priority: Minor
> Fix For: 0.13.0
>
>
> now, when we use sparksql to update hudi table, we find that  hudi will 
> change the nullability attribute of a column
> eg:
> {code:java}
> // code placeholder
>  val tableName = generateTableName
>  val tablePath = s"${new Path(tmp.getCanonicalPath, 
> tableName).toUri.toString}"
>  // create table
>  spark.sql(
>s"""
>   |create table $tableName (
>   |  id int,
>   |  name string,
>   |  price double,
>   |  ts long
>   |) using hudi
>   | location '$tablePath'
>   | options (
>   |  type = '$tableType',
>   |  primaryKey = 'id',
>   |  preCombineField = 'ts'
>   | )
> """.stripMargin)
>  // insert data to table
>  spark.sql(s"insert into $tableName select 1, 'a1', 10, 1000")
>  spark.sql(s"select * from $tableName").printSchema()
>  // update data
>  spark.sql(s"update $tableName set price = 20 where id = 1")
>  spark.sql(s"select * from $tableName").printSchema() {code}
>  
>  |-- _hoodie_commit_time: string (nullable = true)
>  |-- _hoodie_commit_seqno: string (nullable = true)
>  |-- _hoodie_record_key: string (nullable = true)
>  |-- _hoodie_partition_path: string (nullable = true)
>  |-- _hoodie_file_name: string (nullable = true)
>  |-- id: integer (nullable = true)
>  |-- name: string (nullable = true)
>  *|-- price: double (nullable = true)*
>  |-- ts: long (nullable = true)
>  
>  |-- _hoodie_commit_time: string (nullable = true)
>  |-- _hoodie_commit_seqno: string (nullable = true)
>  |-- _hoodie_record_key: string (nullable = true)
>  |-- _hoodie_partition_path: string (nullable = true)
>  |-- _hoodie_file_name: string (nullable = true)
>  |-- id: integer (nullable = true)
>  |-- name: string (nullable = true)
>  *|-- price: double (nullable = false )*
>  |-- ts: long (nullable = true)
>  
> the nullable attribute of price has been changed to false, This is not the 
> result we want



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Updated] (HUDI-3626) Refactor TableSchemaResolver to remove `includeMetadataFields` flags

2022-10-01 Thread Zhaojing Yu (Jira)


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

Zhaojing Yu updated HUDI-3626:
--
Fix Version/s: 0.13.0
   (was: 0.12.1)

> Refactor TableSchemaResolver to remove `includeMetadataFields` flags
> 
>
> Key: HUDI-3626
> URL: https://issues.apache.org/jira/browse/HUDI-3626
> Project: Apache Hudi
>  Issue Type: Bug
>Reporter: Alexey Kudinkin
>Assignee: Alexey Kudinkin
>Priority: Critical
> Fix For: 0.13.0
>
>
> Currently, `TableSchemaResolver` to pass `includeMetadataFields` in its APIs 
> that would selectively remove metadata fields from the returned schemas. 
> There are multiple issues with this flag:
>  # It's applied inconsistently: sometimes it just means that meta fields 
> {_}won't be added{_}, and sometimes it means fields _would be removed_ even 
> if present
>  # This flag actually spells the usage context into TableSchemaResolver: it's 
> highly contextual whether caller wants to remove, omit such meta-fields, or 
> take the schema as is and should not be spilled into the Resolver itself.
>  # B/c of it there's no way to know actually what was actual schema the data 
> was written with (b/c flag might not only omit, but also change the original 
> schema)



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Updated] (HUDI-3649) Add HoodieTableConfig defaults to HoodieWriteConfig

2022-10-01 Thread Zhaojing Yu (Jira)


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

Zhaojing Yu updated HUDI-3649:
--
Fix Version/s: 0.13.0
   (was: 0.12.1)

> Add HoodieTableConfig defaults to HoodieWriteConfig
> ---
>
> Key: HUDI-3649
> URL: https://issues.apache.org/jira/browse/HUDI-3649
> Project: Apache Hudi
>  Issue Type: Improvement
>  Components: configs
>Reporter: Ethan Guo
>Assignee: Sagar Sumit
>Priority: Major
> Fix For: 0.13.0
>
>
> HoodieWriteConfig does not set defaults from HoodieTableConfig.  We need to 
> see if some HoodieTableConfig defaults should be set in HoodieWriteConfig.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Updated] (HUDI-3742) Enable parquet enableVectorizedReader for spark incremental read to prevent pef regression

2022-10-01 Thread Zhaojing Yu (Jira)


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

Zhaojing Yu updated HUDI-3742:
--
Fix Version/s: 0.13.0
   (was: 0.12.1)

> Enable parquet  enableVectorizedReader for spark incremental read to prevent 
> pef regression
> ---
>
> Key: HUDI-3742
> URL: https://issues.apache.org/jira/browse/HUDI-3742
> Project: Apache Hudi
>  Issue Type: Improvement
>  Components: spark
>Reporter: Tao Meng
>Assignee: Tao Meng
>Priority: Critical
>  Labels: pull-request-available
> Fix For: 0.13.0
>
>
> now we disable parquet  enableVectorizedReader for mor incremental read,
> and set "spark.sql.parquet.recordLevelFilter.enabled" = "true"  to achieve 
> data filter
> which is slow



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Updated] (HUDI-3717) Avoid double-listing w/in BaseHoodieTableFileIndex

2022-10-01 Thread Zhaojing Yu (Jira)


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

Zhaojing Yu updated HUDI-3717:
--
Fix Version/s: 0.13.0
   (was: 0.12.1)

> Avoid double-listing w/in BaseHoodieTableFileIndex
> --
>
> Key: HUDI-3717
> URL: https://issues.apache.org/jira/browse/HUDI-3717
> Project: Apache Hudi
>  Issue Type: Bug
>Reporter: Alexey Kudinkin
>Assignee: Alexey Kudinkin
>Priority: Major
> Fix For: 0.13.0
>
> Attachments: Screen Shot 2022-03-25 at 7.05.09 PM.png, Screen Shot 
> 2022-03-25 at 7.05.43 PM.png, Screen Shot 2022-03-25 at 7.14.20 PM.png
>
>
> Currently in `BaseHoodieTableFileIndex::loadPartitionPathFiles` essentially 
> does file-listing twice: 
>  * Once when `getAllQueryPartitionPaths` is invoked
>  * Second time when `getFilesInPartitions` is invoked
>  
> While this will not result in double-listing of the files on FS (b/c of 
> `FIleStatusCache`, if any), this leads however to MT being queried twice: 
> !Screen Shot 2022-03-25 at 7.14.20 PM.png!
>  
> !Screen Shot 2022-03-25 at 7.05.09 PM.png!



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Updated] (HUDI-3818) hudi doesn't support bytes column as primary key

2022-10-01 Thread Zhaojing Yu (Jira)


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

Zhaojing Yu updated HUDI-3818:
--
Fix Version/s: 0.13.0
   (was: 0.12.1)

> hudi doesn't support bytes column as primary key
> 
>
> Key: HUDI-3818
> URL: https://issues.apache.org/jira/browse/HUDI-3818
> Project: Apache Hudi
>  Issue Type: Bug
>  Components: hive
>Reporter: rex xiong
>Assignee: rex xiong
>Priority: Critical
>  Labels: pull-request-available
> Fix For: 0.13.0
>
>
>  when use bytes column as primary key, hudi will generate fixed hoodie key, 
> then upserts will only insert one row. 
> {code:java}
> scala> sql("desc extended binary_test1").show()
> +++---+
> |            col_name|           data_type|comment|
> +++---+
> | _hoodie_commit_time|              string|   null|
> |_hoodie_commit_seqno|              string|   null|
> |  _hoodie_record_key|              string|   null|
> |_hoodie_partition...|              string|   null|
> |   _hoodie_file_name|              string|   null|
> |                  id|              binary|   null|
> |                name|              string|   null|
> |                  dt|              string|   null|
> |                    |                    |       |
> |# Detailed Table ...|                    |       |
> |            Database|             default|       |
> |               Table|        binary_test1|       |
> |               Owner|                root|       |
> |        Created Time|Sat Apr 02 13:28:...|       |
> |         Last Access|             UNKNOWN|       |
> |          Created By|         Spark 3.2.0|       |
> |                Type|             MANAGED|       |
> |            Provider|                hudi|       |
> |    Table Properties|[last_commit_time...|       |
> |          Statistics|        435194 bytes|       |
> +++---+
> scala> sql("select * from binary_test1").show()
> +---+++--+++-++
> |_hoodie_commit_time|_hoodie_commit_seqno|  
> _hoodie_record_key|_hoodie_partition_path|   _hoodie_file_name|               
>    id|     name|      dt|
> +---+++--+++-++
> |  20220402132927590|20220402132927590...|id:java.nio.HeapB...|               
>        |1a06106e-5e7a-4e6...|[03 45 6A 00 00 0...|Mary Jane|20220401|
> +---+++--+++-++{code}



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Updated] (HUDI-3775) Allow for offline compaction of MOR tables via spark streaming

2022-10-01 Thread Zhaojing Yu (Jira)


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

Zhaojing Yu updated HUDI-3775:
--
Fix Version/s: 0.13.0

> Allow for offline compaction of MOR tables via spark streaming
> --
>
> Key: HUDI-3775
> URL: https://issues.apache.org/jira/browse/HUDI-3775
> Project: Apache Hudi
>  Issue Type: Improvement
>  Components: compaction, spark
>Reporter: Rajesh
>Assignee: sivabalan narayanan
>Priority: Critical
>  Labels: easyfix
> Fix For: 0.13.0
>
>
> Currently there is no way to avoid compaction taking up a lot of resources 
> when run inline or async for MOR tables via Spark Streaming. Delta Streamer 
> has ways to assign resources between ingestion and async compaction but Spark 
> Streaming does not have that option. 
> Introducing a flag to turn off automatic compaction and allowing users to run 
> compaction in a separate process will decouple both concerns.
> This will also allow the users to size the cluster just for ingestion and 
> deal with compaction separate without blocking.  We will need to look into 
> documenting best practices for running offline compaction.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Updated] (HUDI-3796) Implement layout to filter out uncommitted log files without reading the log blocks

2022-10-01 Thread Zhaojing Yu (Jira)


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

Zhaojing Yu updated HUDI-3796:
--
Fix Version/s: 0.13.0
   (was: 0.12.1)

> Implement layout to filter out uncommitted log files without reading the log 
> blocks
> ---
>
> Key: HUDI-3796
> URL: https://issues.apache.org/jira/browse/HUDI-3796
> Project: Apache Hudi
>  Issue Type: Improvement
>  Components: writer-core
>Reporter: Ethan Guo
>Assignee: sivabalan narayanan
>Priority: Critical
> Fix For: 0.13.0
>
>
> Related: HUDI-3637
> At high level, getLatestFileSlices() is going to fetch the latest file slices 
> for committed base files and filter out any file slices with the uncommitted 
> base instant time.  The uncommitted log files in the latest file slices may 
> be included, and they are skipped while doing log reading and merging, i.e., 
> the logic in "AbstractHoodieLogRecordReader".
> We can use log instant time instead of base instant time for the log file 
> name so that it is able to filter out uncommitted log files without reading 
> the log blocks beforehand.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[GitHub] [hudi] hudi-bot commented on pull request #6850: [Draft][HUDI-4964] inline all the getter methods that have no logic …

2022-10-01 Thread GitBox


hudi-bot commented on PR #6850:
URL: https://github.com/apache/hudi/pull/6850#issuecomment-1264348973

   
   ## CI report:
   
   * e3aef767db19eed24222f8fff89ae4c59d0799c2 Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11956)
 
   
   
   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] [Updated] (HUDI-3775) Allow for offline compaction of MOR tables via spark streaming

2022-10-01 Thread Zhaojing Yu (Jira)


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

Zhaojing Yu updated HUDI-3775:
--
Fix Version/s: (was: 0.12.1)

> Allow for offline compaction of MOR tables via spark streaming
> --
>
> Key: HUDI-3775
> URL: https://issues.apache.org/jira/browse/HUDI-3775
> Project: Apache Hudi
>  Issue Type: Improvement
>  Components: compaction, spark
>Reporter: Rajesh
>Assignee: sivabalan narayanan
>Priority: Critical
>  Labels: easyfix
>
> Currently there is no way to avoid compaction taking up a lot of resources 
> when run inline or async for MOR tables via Spark Streaming. Delta Streamer 
> has ways to assign resources between ingestion and async compaction but Spark 
> Streaming does not have that option. 
> Introducing a flag to turn off automatic compaction and allowing users to run 
> compaction in a separate process will decouple both concerns.
> This will also allow the users to size the cluster just for ingestion and 
> deal with compaction separate without blocking.  We will need to look into 
> documenting best practices for running offline compaction.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Updated] (HUDI-3881) Implement index syntax for spark sql

2022-10-01 Thread Zhaojing Yu (Jira)


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

Zhaojing Yu updated HUDI-3881:
--
Fix Version/s: 0.13.0
   (was: 0.12.1)

> Implement index syntax for spark sql
> 
>
> Key: HUDI-3881
> URL: https://issues.apache.org/jira/browse/HUDI-3881
> Project: Apache Hudi
>  Issue Type: New Feature
>  Components: spark-sql
>Reporter: Forward Xu
>Assignee: Forward Xu
>Priority: Critical
>  Labels: pull-request-available
> Fix For: 0.13.0
>
>
> {code:java}
> 1. Create Index
> CREATE INDEX [IF NOT EXISTS] index_name
> ON [TABLE] [db_name.]table_name (column_name [ASC|DESC], ...) 
> USING [bloom/lucene]
> [PROPERTIES ('key'='value')] 
> 2. Refresh Index
> REFRESH INDEX index_name ON [TABLE] [db_name.]table_name
> 3. Drop Index 
> DROP INDEX [IF EXISTS] index_name ON [TABLE] [db_name.]table_name
> 4. Show index
> SHOW INDEX [index_name] ON [TABLE] [db_name.]table_name{code}



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Updated] (HUDI-3882) Make sure Hudi Spark relations implementations provide similar file-scanning metrics

2022-10-01 Thread Zhaojing Yu (Jira)


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

Zhaojing Yu updated HUDI-3882:
--
Fix Version/s: 0.13.0
   (was: 0.12.1)

> Make sure Hudi Spark relations implementations provide similar file-scanning 
> metrics
> 
>
> Key: HUDI-3882
> URL: https://issues.apache.org/jira/browse/HUDI-3882
> Project: Apache Hudi
>  Issue Type: Improvement
>Reporter: Alexey Kudinkin
>Assignee: Alexey Kudinkin
>Priority: Critical
> Fix For: 0.13.0
>
>
> After rebasing Hudi's COW table onto `BaseFileOnlyRelation` from 
> `HadoopFsRelation`, we lost very helpful file-scanning metrics like # of 
> files scanned, total size of files scanned, etc.
>  
> After some investigation, i've found out that this occurred b/c now instead 
> of relying on `FileScan` node in Spark plans like we're before, it now uses 
> `DataScan` node that doesn't provide such metrics.
> This unwanted transition occurred b/c Spark internally predicates on 
> `HadoopFsRelation` to decide whether it's a `FileScan` or `DataScan`, and 
> since we stopped using `HadoopFsRelation` Hudi relations now fall into the 
> latter bucket



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Updated] (HUDI-3956) Optimize hudi-presto-bundle size

2022-10-01 Thread Zhaojing Yu (Jira)


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

Zhaojing Yu updated HUDI-3956:
--
Fix Version/s: 0.13.0
   (was: 0.12.1)

> Optimize hudi-presto-bundle size
> 
>
> Key: HUDI-3956
> URL: https://issues.apache.org/jira/browse/HUDI-3956
> Project: Apache Hudi
>  Issue Type: Improvement
>Reporter: Sagar Sumit
>Priority: Major
> Fix For: 0.13.0
>
>
> With the inclusion of hbase-server and its shading, the bundle size is about 
> 33mb compared to 17mb previously. 



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Updated] (HUDI-3919) Support out of order rollback blocks in AbstractHoodieLogRecordReader

2022-10-01 Thread Zhaojing Yu (Jira)


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

Zhaojing Yu updated HUDI-3919:
--
Fix Version/s: 0.13.0
   (was: 0.12.1)

> Support out of order rollback blocks in AbstractHoodieLogRecordReader
> -
>
> Key: HUDI-3919
> URL: https://issues.apache.org/jira/browse/HUDI-3919
> Project: Apache Hudi
>  Issue Type: Improvement
>  Components: writer-core
>Reporter: Surya Prasanna Yalla
>Assignee: Surya Prasanna Yalla
>Priority: Critical
>  Labels: pull-request-available
> Fix For: 0.13.0
>
>
> Currently the rollback block refers to previous data block for invalidating. 
> In multiwriter scenario the block need not be previous block. So, need to 
> support out of order rollback blocks.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Updated] (HUDI-3892) Add HoodieReadClient with java

2022-10-01 Thread Zhaojing Yu (Jira)


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

Zhaojing Yu updated HUDI-3892:
--
Fix Version/s: 0.13.0
   (was: 0.12.1)

> Add HoodieReadClient with java
> --
>
> Key: HUDI-3892
> URL: https://issues.apache.org/jira/browse/HUDI-3892
> Project: Apache Hudi
>  Issue Type: Task
>  Components: reader-core
>Reporter: sivabalan narayanan
>Priority: Critical
> Fix For: 0.13.0
>
>
> We might need a hoodie read client in java similar to the one we have for 
> spark. 
> [Apache Pulsar|https://github.com/apache/pulsar] is doing integration with 
> Hudi, and take Hudi as tiered storage to offload topic cold data into Hudi. 
> When consumers fetch cold data from topic, Pulsar broker will locate the 
> target data is stored in Pulsar or not. If the target data stored in tiered 
> storage (Hudi), Pulsar broker will fetch data from Hudi by Java API, and 
> package them into Pulsar format and dispatch to consumer side.
> However, we found current Hudi implementation doesn't support read Hudi table 
> records by Java API, and we couldn't read the target data out from Hudi into 
> Pulsar Broker, which will block the Pulsar & Hudi integration.
> h3. What we need
>  # We need Hudi to support reading records by Java API
>  # We need Hudi to support read records out which keep the writer order, or 
> support order by specific fields.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Updated] (HUDI-4021) Support deferring compaction when there is an inflight delta commit

2022-10-01 Thread Zhaojing Yu (Jira)


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

Zhaojing Yu updated HUDI-4021:
--
Fix Version/s: 0.13.0
   (was: 0.12.1)

> Support deferring compaction when there is an inflight delta commit
> ---
>
> Key: HUDI-4021
> URL: https://issues.apache.org/jira/browse/HUDI-4021
> Project: Apache Hudi
>  Issue Type: Improvement
>  Components: multi-writer, writer-core
>Reporter: sivabalan narayanan
>Assignee: sivabalan narayanan
>Priority: Major
> Fix For: 0.13.0
>
>
> in multi-writer scenario, if there is an inflight delta commit by writer1 and 
> if writer2 i looking to schedule compaction, the delta commit might 
> eventually fail. This may not be a good experience for end user. Think about 
> regular writes are happening via deltastreamer and a separate async process 
> is taking care of scheduling and executing compaction. Deltastreamer 
> continuous mode might get aborted whenever such overstepping happens. We 
> should add a support where in, compaction scheduling is deferred when an 
> inflight delta commit is found in the timeline. 
>  



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Updated] (HUDI-4036) Investigate whether meta fields could be omitted completely

2022-10-01 Thread Zhaojing Yu (Jira)


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

Zhaojing Yu updated HUDI-4036:
--
Fix Version/s: 0.13.0
   (was: 0.12.1)

> Investigate whether meta fields could be omitted completely
> ---
>
> Key: HUDI-4036
> URL: https://issues.apache.org/jira/browse/HUDI-4036
> Project: Apache Hudi
>  Issue Type: Task
>  Components: writer-core
>Reporter: Alexey Kudinkin
>Assignee: Alexey Kudinkin
>Priority: Critical
> Fix For: 0.13.0
>
>
> Currently, even when meta fields are not populated, we still insert 
> empty-string columns to adhere to the expected schema.
> This has a non-trivial overhead of ~20% (relative to just writing dataset as 
> is), since Spark had to essentially "re-write" the original row with 
> prepended new fields.
> We should investigate whether it's feasible to avoid adding empty-string 
> columns completely if meta-fields are disabled.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Updated] (HUDI-4010) DynamoDB lock configs for naming/docs could be improved

2022-10-01 Thread Zhaojing Yu (Jira)


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

Zhaojing Yu updated HUDI-4010:
--
Status: Patch Available  (was: In Progress)

> DynamoDB lock configs for naming/docs could be improved
> ---
>
> Key: HUDI-4010
> URL: https://issues.apache.org/jira/browse/HUDI-4010
> Project: Apache Hudi
>  Issue Type: Improvement
>Reporter: Udit Mehrotra
>Assignee: Ethan Guo
>Priority: Major
>  Labels: pull-request-available
> Fix For: 0.12.1
>
>
> The issue was raised in [https://github.com/apache/hudi/issues/5451,] where 
> the customer created the DynamoDB table manually and assumed *partition_key* 
> property to be the column name that is to be used instead of the actual value 
> of the partition key column. The name of the partition key column is fixed in 
> code to {*}key{*}.
>  
> There is scope to improve the naming, so it is not confusing whether one 
> needs to define a columns name there or the column value to be used for that 
> specific table. Also may be we can better document that the table can be 
> created automatically by Hudi code, and if at all someone creates table by 
> themselves, what column name to use.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Updated] (HUDI-4209) TestHoodieBulkInsertDataInternalWriter fails due to MiniDFSCluster class not found

2022-10-01 Thread Zhaojing Yu (Jira)


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

Zhaojing Yu updated HUDI-4209:
--
Fix Version/s: 0.13.0
   (was: 0.12.1)

> TestHoodieBulkInsertDataInternalWriter fails due to MiniDFSCluster class not 
> found
> --
>
> Key: HUDI-4209
> URL: https://issues.apache.org/jira/browse/HUDI-4209
> Project: Apache Hudi
>  Issue Type: Task
>Reporter: Sagar Sumit
>Priority: Critical
> Fix For: 0.13.0
>
>




--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Updated] (HUDI-4184) Creating external table in Spark SQL modifies "hoodie.properties"

2022-10-01 Thread Zhaojing Yu (Jira)


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

Zhaojing Yu updated HUDI-4184:
--
Fix Version/s: 0.13.0
   (was: 0.12.1)

> Creating external table in Spark SQL modifies "hoodie.properties"
> -
>
> Key: HUDI-4184
> URL: https://issues.apache.org/jira/browse/HUDI-4184
> Project: Apache Hudi
>  Issue Type: Bug
>  Components: spark-sql
>Reporter: Alexey Kudinkin
>Assignee: Sagar Sumit
>Priority: Critical
> Fix For: 0.13.0
>
>
> My setup was like following:
>  # There's a table existing in one AWS account
>  # I'm trying to access that table from Spark SQL from _another_ AWS account 
> that only has Read permissions to the bucket with the table.
>  # Now when issuing "CREATE TABLE" Spark SQL command it fails b/c Hudi tries 
> to modify "hoodie.properties" file for whatever reason, even though i'm not 
> modifying the table and just trying to create table in the catalog.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Updated] (HUDI-4123) HoodieDeltaStreamer throws exception due to SqlSource return null checkpoint

2022-10-01 Thread Zhaojing Yu (Jira)


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

Zhaojing Yu updated HUDI-4123:
--
Fix Version/s: 0.13.0
   (was: 0.12.1)

> HoodieDeltaStreamer throws exception due to SqlSource return null checkpoint
> 
>
> Key: HUDI-4123
> URL: https://issues.apache.org/jira/browse/HUDI-4123
> Project: Apache Hudi
>  Issue Type: Bug
>  Components: deltastreamer
>Reporter: 董可伦
>Assignee: 董可伦
>Priority: Critical
>  Labels: pull-request-available
> Fix For: 0.13.0
>
>
> When use SqlSource:
> ## Create hive source table
> ```sql
> create database test location '/test';
> create table test.test_source (
>   id int,
>   name string,
>   price double,
>   dt string,
>   ts bigint
> );
> insert into test.test_source values (105,'hudi', 10.0,'2021-05-05',100);
> ```
> ## Use SqlSource
> sql_source.properties
> ```
> hoodie.datasource.write.recordkey.field=id
> hoodie.datasource.write.partitionpath.field=dt
> hoodie.deltastreamer.source.sql.sql.query = select * from test.test_source
> hoodie.datasource.hive_sync.table=test_hudi_target
> hoodie.datasource.hive_sync.database=hudi
> hoodie.datasource.hive_sync.partition_fields=dt
> hoodie.datasource.hive_sync.create_managed_table = true
> hoodie.datasource.write.hive_style_partitioning=true
> hoodie.datasource.write.keygenerator.class=org.apache.hudi.keygen.ComplexKeyGenerator
> hoodie.datasource.hive_sync.use_jdbc=false
> hoodie.datasource.hive_sync.partition_extractor_class=org.apache.hudi.hive.MultiPartKeysValueExtractor
> ```
> ```bash
> spark-submit --conf "spark.sql.catalogImplementation=hive" \
> --master yarn --deploy-mode client --executor-memory 2G --num-executors 3 
> --executor-cores 2 --driver-memory 4G --driver-cores 2 \
> --principal spark/indata-10-110-105-163.indata@indata.com --keytab 
> /etc/security/keytabs/spark.service.keytab \
> --class org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer 
> /usr/hdp/3.1.0.0-78/spark2/jars/hudi-utilities-bundle_2.11-0.12.0-SNAPSHOT.jar
>  \
> --props file:///opt/sql_source.properties  \
> --target-base-path /hudi/test_hudi_target \
> --target-table test_hudi_target \
> --op BULK_INSERT \
> --table-type COPY_ON_WRITE \
> --source-ordering-field ts \
> --source-class org.apache.hudi.utilities.sources.SqlSource \
> --enable-sync  \
> --checkpoint earliest \
> --allow-commit-on-no-checkpoint-change
> ```
> Once executed, the hive source table can be successfully written to the Hudi 
> target table.
> However, if it is executed multiple times, such as the second time, an 
> exception will be thrown:
> ```
> org.apache.hudi.utilities.exception.HoodieDeltaStreamerException: Unable to 
> find previous checkpoint. Please double check if this table was indeed built 
> via delta streamer. Last Commit :
> "deltastreamer.checkpoint.reset_key" : "earliest"
> ```
> The reason is that the value of `deltastreamer.checkpoint.reset_key` is 
> `earliest`,but `deltastreamer.checkpoint.key` is null,
> According to the logic of the method `getCheckpointToResume`,Will throw this 
> exception.
> I think since  the value of `deltastreamer.checkpoint.reset_key` is null, The 
> value of `deltastreamer.checkpoint.key`should also be saved as null.This also 
> avoids this exception according to the logic of the method 
> `getCheckpointToResume`
>  
>  
> org.apache.hudi.utilities.exception.HoodieDeltaStreamerException: Unable to 
> find previous checkpoint. Please double check if this table was indeed built 
> via delta streamer. Last Commit 
> :Option{val=[20220519162403646__commit__COMPLETED]}, Instants 
> :[[20220519162403646__commit__COMPLETED]], CommitMetadata={
>   "partitionToWriteStats" : {
>     "2016/03/15" : [
> {       "fileId" : "6a1e0512-508a-4bdb-ad8f-200cda157ff0-0",       "path" : 
> "2016/03/15/6a1e0512-508a-4bdb-ad8f-200cda157ff0-0_0-21-21_20220519162403646.parquet",
>        "prevCommit" : "null",       "numWrites" : 342,       "numDeletes" : 
> 0,       "numUpdateWrites" : 0,       "numInserts" : 342,       
> "totalWriteBytes" : 481336,       "totalWriteErrors" : 0,       "tempPath" : 
> null,       "partitionPath" : "2016/03/15",       "totalLogRecords" : 0,      
>  "totalLogFilesCompacted" : 0,       "totalLogSizeCompacted" : 0,       
> "totalUpdatedRecordsCompacted" : 0,       "totalLogBlocks" : 0,       
> "totalCorruptLogBlock" : 0,       "totalRollbackBlocks" : 0,       
> "fileSizeInBytes" : 481336,       "minEventTime" : null,       "maxEventTime" 
> : null     }
> ],
>     "2015/03/16" : [
> {       "fileId" : "f3371308-8809-4644-baf6-c65c3fb86c8e-0",       "path" : 
> "2015/03/16/f3371308-8809-4644-baf6-c65c3fb86c8e-0_1-21-22_20220519162403646.parquet",
>        "prevCommit" : "null",       

[jira] [Resolved] (HUDI-4237) spark.sql.sources.schema.partCol.0 is non-empty in HiveMetaStore when create non-partition hudi table in Spark

2022-10-01 Thread Zhaojing Yu (Jira)


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

Zhaojing Yu resolved HUDI-4237.
---

> spark.sql.sources.schema.partCol.0 is non-empty in HiveMetaStore when create 
> non-partition hudi table in Spark
> --
>
> Key: HUDI-4237
> URL: https://issues.apache.org/jira/browse/HUDI-4237
> Project: Apache Hudi
>  Issue Type: Bug
>Reporter: du.junling
>Priority: Critical
>  Labels: pull-request-available
> Fix For: 0.12.1
>
>
> Create a non-partition hudi table in Spark,it will store 
> spark.sql.sources.schema.partCol.0 with an empty value in hiveMetastore.
> This is unexpected behavior.
> Steps to reproduce the behavior:
> 1. Create a non-partition hudi table in Spark
> {code:java}
> create table hudi_mor_tbl (
> id int,
> name string,
> price double,
> ts bigint
> ) using hudi
> tblproperties (
> type = 'mor',
> primaryKey = 'id',
> preCombineField = 'ts'
> ) {code}
> 2. insert data one row to it.
> {code:java}
> insert into hudi_mor_tbl select 1, 'a1', 20, 1000; {code}
> 3. cat hoodie.properties in table's base path,
> it include partition.fields key with an empty value
> {code:java}
> hoodie.table.partition.fields=
>  {code}
> 4. check spark.sql.sources.schema.partCol.0 that stored in table TABLE_PARAMS 
> of the HiveMetaStore .
> {code:java}
> |50|spark.sql.sources.schema.partCol.0|
>  {code}
> it has a value "".
> *Expected behavior*
> this is no hoodie.table.partition.fields in hoodie.properties and 
> spark.sql.sources.schema.partCol.0  in HiveMetastore
> *Environment Description*
>  * Hudi version : 0.10.0
>  * Spark version : 3.2.1
>  * Hive version : 3.1.2
>  * Hadoop version : 3.3.1
>  * Storage (HDFS/S3/GCS..) : HDFS
>  * Running on Docker? (yes/no) : no



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Updated] (HUDI-4330) NPE when trying to upsert into a dataset with no Meta Fields

2022-10-01 Thread Zhaojing Yu (Jira)


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

Zhaojing Yu updated HUDI-4330:
--
Fix Version/s: 0.13.0

> NPE when trying to upsert into a dataset with no Meta Fields
> 
>
> Key: HUDI-4330
> URL: https://issues.apache.org/jira/browse/HUDI-4330
> Project: Apache Hudi
>  Issue Type: Bug
>Reporter: Alexey Kudinkin
>Assignee: Raymond Xu
>Priority: Critical
> Fix For: 0.13.0
>
>
> When trying to upsert into a dataset with Meta Fields being disabled, you 
> will encounter obscure NPE like below:
> {code:java}
> Caused by: org.apache.spark.SparkException: Job aborted due to stage failure: 
> Task 25 in stage 20.0 failed 4 times, most recent failure: Lost task 25.3 in 
> stage 20.0 (TID 4110) (ip-172-31-20-53.us-west-2.compute.internal executor 
> 7): java.lang.RuntimeException: 
> org.apache.hudi.exception.HoodieIndexException: Error checking bloom filter 
> index.
>         at 
> org.apache.hudi.client.utils.LazyIterableIterator.next(LazyIterableIterator.java:121)
>         at 
> scala.collection.convert.Wrappers$JIteratorWrapper.next(Wrappers.scala:46)
>         at scala.collection.Iterator$$anon$11.nextCur(Iterator.scala:486)
>         at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:492)
>         at scala.collection.Iterator$$anon$12.hasNext(Iterator.scala:513)
>         at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:491)
>         at 
> org.apache.spark.shuffle.sort.BypassMergeSortShuffleWriter.write(BypassMergeSortShuffleWriter.java:140)
>         at 
> org.apache.spark.shuffle.ShuffleWriteProcessor.write(ShuffleWriteProcessor.scala:59)
>         at 
> org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:99)
>         at 
> org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:52)
>         at org.apache.spark.scheduler.Task.run(Task.scala:131)
>         at 
> org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:506)
>         at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1462)
>         at 
> org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:509)
>         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:750)
> Caused by: org.apache.hudi.exception.HoodieIndexException: Error checking 
> bloom filter index.
>         at 
> org.apache.hudi.index.bloom.HoodieBloomIndexCheckFunction$LazyKeyCheckIterator.computeNext(HoodieBloomIndexCheckFunction.java:110)
>         at 
> org.apache.hudi.index.bloom.HoodieBloomIndexCheckFunction$LazyKeyCheckIterator.computeNext(HoodieBloomIndexCheckFunction.java:60)
>         at 
> org.apache.hudi.client.utils.LazyIterableIterator.next(LazyIterableIterator.java:119)
>         ... 16 more
> Caused by: java.lang.NullPointerException
>         at 
> org.apache.hudi.io.HoodieKeyLookupHandle.addKey(HoodieKeyLookupHandle.java:88)
>         at 
> org.apache.hudi.index.bloom.HoodieBloomIndexCheckFunction$LazyKeyCheckIterator.computeNext(HoodieBloomIndexCheckFunction.java:92)
>         ... 18 more {code}
> Instead, we could be more explicit as to why this could have happened 
> (meta-fields disabled -> no bloom filter created -> unable to do upserts)



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Updated] (HUDI-4244) Support common Spark transformations w/in Spark SQL "partitioned by" clause

2022-10-01 Thread Zhaojing Yu (Jira)


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

Zhaojing Yu updated HUDI-4244:
--
Fix Version/s: 0.13.0
   (was: 0.12.1)

> Support common Spark transformations w/in Spark SQL "partitioned by" clause
> ---
>
> Key: HUDI-4244
> URL: https://issues.apache.org/jira/browse/HUDI-4244
> Project: Apache Hudi
>  Issue Type: Improvement
>Reporter: Alexey Kudinkin
>Priority: Major
> Fix For: 0.13.0
>
>
> Currently if you create a Hudi table from Spark SQL:
> {code:java}
> CREATE TABLE test_create(
> f1 STRING,
> f2 STRING,
> f3 STRING,
> ts timestamp
> )using hudi
>  partitioned by ( hours(ts))
>  options (
>   type = 'mor'
>  )
>  tblproperties (
>   primaryKey = 'f1',
>   preCombineField = 'ts'
> );
>  {code}
> You'll be getting
> {code:java}
> java.util.concurrent.ExecutionException: java.lang.RuntimeException: 
> org.apache.spark.sql.AnalysisException: Transforms cannot be converted to 
> partition columns: hours(ts){code}
>  
> Original reported task:
> [https://github.com/apache/hudi/issues/5810]
>  



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Updated] (HUDI-4302) CI Instability / flaky tests

2022-10-01 Thread Zhaojing Yu (Jira)


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

Zhaojing Yu updated HUDI-4302:
--
Fix Version/s: 0.13.0
   (was: 0.12.1)

> CI Instability / flaky tests
> 
>
> Key: HUDI-4302
> URL: https://issues.apache.org/jira/browse/HUDI-4302
> Project: Apache Hudi
>  Issue Type: Epic
>  Components: tests-ci
>Reporter: sivabalan narayanan
>Priority: Major
> Fix For: 0.13.0
>
>
> Creating an EPIC to track the flaky tests



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Updated] (HUDI-4457) Make sure IT docker test return code non-zero when failed

2022-10-01 Thread Zhaojing Yu (Jira)


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

Zhaojing Yu updated HUDI-4457:
--
Fix Version/s: 0.13.0
   (was: 0.12.1)

> Make sure IT docker test return code non-zero when failed
> -
>
> Key: HUDI-4457
> URL: https://issues.apache.org/jira/browse/HUDI-4457
> Project: Apache Hudi
>  Issue Type: Bug
>  Components: tests-ci
>Reporter: Raymond Xu
>Priority: Critical
> Fix For: 0.13.0
>
>
> IT testcase where docker command runs and returns exit code 0, but test 
> actually failed. This will be misleading for troubleshooting.
> TODO
> 1. verify the behavior
> 2. fix it



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Updated] (HUDI-4542) Flink streaming query fails with ClassNotFoundException

2022-10-01 Thread Zhaojing Yu (Jira)


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

Zhaojing Yu updated HUDI-4542:
--
Fix Version/s: 0.13.0
   (was: 0.12.1)

> Flink streaming query fails with ClassNotFoundException
> ---
>
> Key: HUDI-4542
> URL: https://issues.apache.org/jira/browse/HUDI-4542
> Project: Apache Hudi
>  Issue Type: Bug
>  Components: flink-sql
>Reporter: Ethan Guo
>Priority: Critical
> Fix For: 0.13.0
>
> Attachments: Screen Shot 2022-08-04 at 17.17.42.png
>
>
> Environment: EMR 6.7.0 Flink 1.14.2
> Reproducible steps: Build Hudi Flink bundle from master
> {code:java}
> mvn clean package -DskipTests  -pl :hudi-flink1.14-bundle -am {code}
> Copy to EMR master node /lib/flink/lib
> Launch Flink SQL client:
> {code:java}
> cd /lib/flink && ./bin/yarn-session.sh --detached
> ./bin/sql-client.sh {code}
> Write a Hudi table with a few commits with metadata table enabled (no column 
> stats).  Then, run the following for the streaming query
> {code:java}
> CREATE TABLE t2(
>    uuid VARCHAR(20) PRIMARY KEY NOT ENFORCED,
>    name VARCHAR(10),
>    age INT,
>    ts TIMESTAMP(3),
>    `partition` VARCHAR(20)
>  )
>  PARTITIONED BY (`partition`)
>  WITH (
>    'connector' = 'hudi',
>    'path' = 's3a://',
>    'table.type' = 'MERGE_ON_READ',
>    'read.streaming.enabled' = 'true',  -- this option enable the streaming 
> read
>    'read.start-commit' = '20220803165232362', -- specifies the start commit 
> instant time
>    'read.streaming.check-interval' = '4' -- specifies the check interval for 
> finding new source commits, default 60s.
>  ); {code}
> {code:java}
> select * from t2; {code}
> {code:java}
> Flink SQL> select * from t2;
> 2022-08-05 00:12:43,635 INFO  org.apache.hadoop.metrics2.impl.MetricsConfig   
>              [] - Loaded properties from hadoop-metrics2.properties
> 2022-08-05 00:12:43,650 INFO  
> org.apache.hadoop.metrics2.impl.MetricsSystemImpl            [] - Scheduled 
> Metric snapshot period at 300 second(s).
> 2022-08-05 00:12:43,650 INFO  
> org.apache.hadoop.metrics2.impl.MetricsSystemImpl            [] - 
> s3a-file-system metrics system started
> 2022-08-05 00:12:47,722 INFO  org.apache.hadoop.fs.s3a.S3AInputStream         
>              [] - Switching to Random IO seek policy
> 2022-08-05 00:12:47,941 INFO  org.apache.hadoop.yarn.client.RMProxy           
>              [] - Connecting to ResourceManager at 
> ip-172-31-9-157.us-east-2.compute.internal/172.31.9.157:8032
> 2022-08-05 00:12:47,942 INFO  org.apache.hadoop.yarn.client.AHSProxy          
>              [] - Connecting to Application History server at 
> ip-172-31-9-157.us-east-2.compute.internal/172.31.9.157:10200
> 2022-08-05 00:12:47,942 INFO  org.apache.flink.yarn.YarnClusterDescriptor     
>              [] - No path for the flink jar passed. Using the location of 
> class org.apache.flink.yarn.YarnClusterDescriptor to locate the jar
> 2022-08-05 00:12:47,942 WARN  org.apache.flink.yarn.YarnClusterDescriptor     
>              [] - Neither the HADOOP_CONF_DIR nor the YARN_CONF_DIR 
> environment variable is set.The Flink YARN Client needs one of these to be 
> set to properly load the Hadoop configuration for accessing YARN.
> 2022-08-05 00:12:47,959 INFO  org.apache.flink.yarn.YarnClusterDescriptor     
>              [] - Found Web Interface 
> ip-172-31-3-92.us-east-2.compute.internal:39605 of application 
> 'application_1659656614768_0001'.
> [ERROR] Could not execute SQL statement. Reason:
> java.lang.ClassNotFoundException: 
> org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat{code}
> {code:java}
> 2022-08-04 17:12:59
> org.apache.flink.runtime.JobException: Recovery is suppressed by 
> NoRestartBackoffTimeStrategy
>     at 
> org.apache.flink.runtime.executiongraph.failover.flip1.ExecutionFailureHandler.handleFailure(ExecutionFailureHandler.java:138)
>     at 
> org.apache.flink.runtime.executiongraph.failover.flip1.ExecutionFailureHandler.getFailureHandlingResult(ExecutionFailureHandler.java:82)
>     at 
> org.apache.flink.runtime.scheduler.DefaultScheduler.handleTaskFailure(DefaultScheduler.java:228)
>     at 
> org.apache.flink.runtime.scheduler.DefaultScheduler.maybeHandleTaskFailure(DefaultScheduler.java:218)
>     at 
> org.apache.flink.runtime.scheduler.DefaultScheduler.updateTaskExecutionStateInternal(DefaultScheduler.java:209)
>     at 
> org.apache.flink.runtime.scheduler.SchedulerBase.updateTaskExecutionState(SchedulerBase.java:679)
>     at 
> org.apache.flink.runtime.scheduler.SchedulerNG.updateTaskExecutionState(SchedulerNG.java:79)
>     at 
> org.apache.flink.runtime.jobmaster.JobMaster.updateTaskExecutionState(JobMaster.java:444)
>     at sun.reflect.GeneratedMethodAccessor35.invoke(Unknown Source)
>     at 
> 

[jira] [Resolved] (HUDI-4488) Improve S3 File listing efficiency

2022-10-01 Thread Zhaojing Yu (Jira)


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

Zhaojing Yu resolved HUDI-4488.
---

> Improve S3 File listing efficiency
> --
>
> Key: HUDI-4488
> URL: https://issues.apache.org/jira/browse/HUDI-4488
> Project: Apache Hudi
>  Issue Type: Sub-task
>Reporter: Vamshi Gudavarthi
>Priority: Major
>  Labels: pull-request-available
> Fix For: 0.12.1
>
>
> Improve S3 file listing my moving the check to executor from driver.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Updated] (HUDI-4522) [DOCS] Set presto session prop to use parquet column names in case of type mismatch

2022-10-01 Thread Zhaojing Yu (Jira)


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

Zhaojing Yu updated HUDI-4522:
--
Fix Version/s: 0.13.0
   (was: 0.12.1)

> [DOCS] Set presto session prop to use parquet column names in case of type 
> mismatch
> ---
>
> Key: HUDI-4522
> URL: https://issues.apache.org/jira/browse/HUDI-4522
> Project: Apache Hudi
>  Issue Type: Task
>Reporter: Sagar Sumit
>Assignee: Léo Biscassi
>Priority: Major
>  Labels: pull-request-available
> Fix For: 0.13.0
>
>
> See https://github.com/apache/hudi/issues/6142



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[GitHub] [hudi] hudi-bot commented on pull request #6850: [Draft][HUDI-4964] inline all the getter methods that have no logic …

2022-10-01 Thread GitBox


hudi-bot commented on PR #6850:
URL: https://github.com/apache/hudi/pull/6850#issuecomment-1264347926

   
   ## CI report:
   
   * e3aef767db19eed24222f8fff89ae4c59d0799c2 Azure: 
[PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11956)
 
   
   
   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 #6848: [HUDI-4769] Option read.streaming.skip_compaction skips delta commit

2022-10-01 Thread GitBox


hudi-bot commented on PR #6848:
URL: https://github.com/apache/hudi/pull/6848#issuecomment-1264347911

   
   ## CI report:
   
   * 6318c29ab4e0415d365db365aac92f1f562a4b68 Azure: 
[CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11954)
 
   * fcfc1baee8aa5394c4472135f6ceb0d280cf8c16 Azure: 
[PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11955)
 
   
   
   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] [Updated] (HUDI-4330) NPE when trying to upsert into a dataset with no Meta Fields

2022-10-01 Thread Zhaojing Yu (Jira)


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

Zhaojing Yu updated HUDI-4330:
--
Fix Version/s: (was: 0.12.1)

> NPE when trying to upsert into a dataset with no Meta Fields
> 
>
> Key: HUDI-4330
> URL: https://issues.apache.org/jira/browse/HUDI-4330
> Project: Apache Hudi
>  Issue Type: Bug
>Reporter: Alexey Kudinkin
>Assignee: Raymond Xu
>Priority: Critical
>
> When trying to upsert into a dataset with Meta Fields being disabled, you 
> will encounter obscure NPE like below:
> {code:java}
> Caused by: org.apache.spark.SparkException: Job aborted due to stage failure: 
> Task 25 in stage 20.0 failed 4 times, most recent failure: Lost task 25.3 in 
> stage 20.0 (TID 4110) (ip-172-31-20-53.us-west-2.compute.internal executor 
> 7): java.lang.RuntimeException: 
> org.apache.hudi.exception.HoodieIndexException: Error checking bloom filter 
> index.
>         at 
> org.apache.hudi.client.utils.LazyIterableIterator.next(LazyIterableIterator.java:121)
>         at 
> scala.collection.convert.Wrappers$JIteratorWrapper.next(Wrappers.scala:46)
>         at scala.collection.Iterator$$anon$11.nextCur(Iterator.scala:486)
>         at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:492)
>         at scala.collection.Iterator$$anon$12.hasNext(Iterator.scala:513)
>         at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:491)
>         at 
> org.apache.spark.shuffle.sort.BypassMergeSortShuffleWriter.write(BypassMergeSortShuffleWriter.java:140)
>         at 
> org.apache.spark.shuffle.ShuffleWriteProcessor.write(ShuffleWriteProcessor.scala:59)
>         at 
> org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:99)
>         at 
> org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:52)
>         at org.apache.spark.scheduler.Task.run(Task.scala:131)
>         at 
> org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:506)
>         at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1462)
>         at 
> org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:509)
>         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:750)
> Caused by: org.apache.hudi.exception.HoodieIndexException: Error checking 
> bloom filter index.
>         at 
> org.apache.hudi.index.bloom.HoodieBloomIndexCheckFunction$LazyKeyCheckIterator.computeNext(HoodieBloomIndexCheckFunction.java:110)
>         at 
> org.apache.hudi.index.bloom.HoodieBloomIndexCheckFunction$LazyKeyCheckIterator.computeNext(HoodieBloomIndexCheckFunction.java:60)
>         at 
> org.apache.hudi.client.utils.LazyIterableIterator.next(LazyIterableIterator.java:119)
>         ... 16 more
> Caused by: java.lang.NullPointerException
>         at 
> org.apache.hudi.io.HoodieKeyLookupHandle.addKey(HoodieKeyLookupHandle.java:88)
>         at 
> org.apache.hudi.index.bloom.HoodieBloomIndexCheckFunction$LazyKeyCheckIterator.computeNext(HoodieBloomIndexCheckFunction.java:92)
>         ... 18 more {code}
> Instead, we could be more explicit as to why this could have happened 
> (meta-fields disabled -> no bloom filter created -> unable to do upserts)



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Updated] (HUDI-4454) Support hiveSync command based on Call Procedure Command

2022-10-01 Thread Zhaojing Yu (Jira)


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

Zhaojing Yu updated HUDI-4454:
--
Fix Version/s: 0.13.0
   (was: 0.12.1)

> Support hiveSync command based on Call Procedure Command
> 
>
> Key: HUDI-4454
> URL: https://issues.apache.org/jira/browse/HUDI-4454
> Project: Apache Hudi
>  Issue Type: New Feature
>  Components: spark-sql
>Reporter: Forward Xu
>Assignee: Forward Xu
>Priority: Major
> Fix For: 0.13.0
>
>




--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Updated] (HUDI-4341) HoodieHFileReader is not compatible with Hadoop 3

2022-10-01 Thread Zhaojing Yu (Jira)


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

Zhaojing Yu updated HUDI-4341:
--
Fix Version/s: 0.13.0
   (was: 0.12.1)

> HoodieHFileReader is not compatible with Hadoop 3
> -
>
> Key: HUDI-4341
> URL: https://issues.apache.org/jira/browse/HUDI-4341
> Project: Apache Hudi
>  Issue Type: Bug
>  Components: metadata
>Reporter: Ethan Guo
>Assignee: Ethan Guo
>Priority: Blocker
>  Labels: spark
> Fix For: 0.13.0
>
>
> [https://github.com/apache/hudi/issues/5765]
> Spark SQL throws "java.lang.NoSuchMethodError: 
> org.apache.hadoop.hdfs.client.HdfsDataInputStream.getReadStatistics()" after 
> a while.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Updated] (HUDI-4369) Hudi Kafka Connect Sink writing to GCS bucket

2022-10-01 Thread Zhaojing Yu (Jira)


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

Zhaojing Yu updated HUDI-4369:
--
Fix Version/s: 0.13.0
   (was: 0.12.1)

> Hudi Kafka Connect Sink writing to GCS bucket
> -
>
> Key: HUDI-4369
> URL: https://issues.apache.org/jira/browse/HUDI-4369
> Project: Apache Hudi
>  Issue Type: Bug
>  Components: kafka-connect
>Reporter: Vishal Agarwal
>Priority: Critical
> Fix For: 0.13.0
>
>
> Hi team,
> I am trying to use Hudi sink connector with Kafka Connect to write to GCS 
> bucket. But I am getting error regarding "gs" file scheme. I have added all 
> GCS related properties in core-site.xml and the corresponding gcs-connector 
> jar in the plugin path. But still facing the issue.
> The issue was already reported with S3 as per jira 
> https://issues.apache.org/jira/browse/HUDI-3610. But I am unable to get the 
> resolution.
> Happy to discuss on this !
> Thanks
> *StackTrace-*
> %d [%thread] %-5level %logger - %msg%n 
> org.apache.hudi.exception.HoodieException: Fatal error instantiating Hudi 
> Write Provider 
>  at 
> org.apache.hudi.connect.writers.KafkaConnectWriterProvider.(KafkaConnectWriterProvider.java:103)
>  ~[connectors-uber.jar:?]
>  at 
> org.apache.hudi.connect.transaction.ConnectTransactionParticipant.(ConnectTransactionParticipant.java:65)
>  ~[connectors-uber.jar:?]
>  at org.apache.hudi.connect.HoodieSinkTask.bootstrap(HoodieSinkTask.java:198) 
> [connectors-uber.jar:?]
>  at org.apache.hudi.connect.HoodieSinkTask.open(HoodieSinkTask.java:151) 
> [connectors-uber.jar:?]
>  at 
> org.apache.kafka.connect.runtime.WorkerSinkTask.openPartitions(WorkerSinkTask.java:587)
>  [connect-runtime-2.4.1.jar:?]
>  at 
> org.apache.kafka.connect.runtime.WorkerSinkTask.access$1100(WorkerSinkTask.java:67)
>  [connect-runtime-2.4.1.jar:?]
>  at 
> org.apache.kafka.connect.runtime.WorkerSinkTask$HandleRebalance.onPartitionsAssigned(WorkerSinkTask.java:652)
>  [connect-runtime-2.4.1.jar:?]
>  at 
> org.apache.kafka.clients.consumer.internals.ConsumerCoordinator.invokePartitionsAssigned(ConsumerCoordinator.java:272)
>  [kafka-clients-2.4.1.jar:?]
>  at 
> org.apache.kafka.clients.consumer.internals.ConsumerCoordinator.onJoinComplete(ConsumerCoordinator.java:400)
>  [kafka-clients-2.4.1.jar:?]
>  at 
> org.apache.kafka.clients.consumer.internals.AbstractCoordinator.joinGroupIfNeeded(AbstractCoordinator.java:421)
>  [kafka-clients-2.4.1.jar:?]
>  at 
> org.apache.kafka.clients.consumer.internals.AbstractCoordinator.ensureActiveGroup(AbstractCoordinator.java:340)
>  [kafka-clients-2.4.1.jar:?]
>  at 
> org.apache.kafka.clients.consumer.internals.ConsumerCoordinator.poll(ConsumerCoordinator.java:471)
>  [kafka-clients-2.4.1.jar:?]
>  at 
> org.apache.kafka.clients.consumer.KafkaConsumer.updateAssignmentMetadataIfNeeded(KafkaConsumer.java:1267)
>  [kafka-clients-2.4.1.jar:?]
>  at 
> org.apache.kafka.clients.consumer.KafkaConsumer.poll(KafkaConsumer.java:1231) 
> [kafka-clients-2.4.1.jar:?]
>  at 
> org.apache.kafka.clients.consumer.KafkaConsumer.poll(KafkaConsumer.java:1211) 
> [kafka-clients-2.4.1.jar:?]
>  at 
> org.apache.kafka.connect.runtime.WorkerSinkTask.pollConsumer(WorkerSinkTask.java:444)
>  [connect-runtime-2.4.1.jar:?]
>  at 
> org.apache.kafka.connect.runtime.WorkerSinkTask.poll(WorkerSinkTask.java:317) 
> [connect-runtime-2.4.1.jar:?]
>  at 
> org.apache.kafka.connect.runtime.WorkerSinkTask.iteration(WorkerSinkTask.java:224)
>  [connect-runtime-2.4.1.jar:?]
>  at 
> org.apache.kafka.connect.runtime.WorkerSinkTask.execute(WorkerSinkTask.java:192)
>  [connect-runtime-2.4.1.jar:?]
>  at org.apache.kafka.connect.runtime.WorkerTask.doRun(WorkerTask.java:177) 
> [connect-runtime-2.4.1.jar:?]
>  at org.apache.kafka.connect.runtime.WorkerTask.run(WorkerTask.java:227) 
> [connect-runtime-2.4.1.jar:?]
>  at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) 
> [?:1.8.0_331]
>  at java.util.concurrent.FutureTask.run(FutureTask.java:266) [?:1.8.0_331]
>  at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
>  [?:1.8.0_331]
>  at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
>  [?:1.8.0_331]
>  at java.lang.Thread.run(Thread.java:750) [?:1.8.0_331]
> Caused by: org.apache.hudi.exception.HoodieIOException: Failed to get 
> instance of org.apache.hadoop.fs.FileSystem
>  at org.apache.hudi.common.fs.FSUtils.getFs(FSUtils.java:109) 
> ~[connectors-uber.jar:?]
>  at org.apache.hudi.common.fs.FSUtils.getFs(FSUtils.java:100) 
> ~[connectors-uber.jar:?]
>  at org.apache.hudi.client.BaseHoodieClient.(BaseHoodieClient.java:69) 
> ~[connectors-uber.jar:?]
>  at 
> org.apache.hudi.client.BaseHoodieWriteClient.(BaseHoodieWriteClient.java:175)
>  ~[connectors-uber.jar:?]

[jira] [Updated] (HUDI-4573) Fix HoodieMultiTableDeltaStreamer to write all tables in continuous mode

2022-10-01 Thread Zhaojing Yu (Jira)


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

Zhaojing Yu updated HUDI-4573:
--
Fix Version/s: 0.13.0
   (was: 0.12.1)

> Fix HoodieMultiTableDeltaStreamer to write all tables in continuous mode
> 
>
> Key: HUDI-4573
> URL: https://issues.apache.org/jira/browse/HUDI-4573
> Project: Apache Hudi
>  Issue Type: Bug
>  Components: deltastreamer
>Reporter: Ethan Guo
>Assignee: sivabalan narayanan
>Priority: Blocker
> Fix For: 0.13.0
>
>




--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Updated] (HUDI-4624) Make sure all DeltaStreamer Sources are Closeable

2022-10-01 Thread Zhaojing Yu (Jira)


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

Zhaojing Yu updated HUDI-4624:
--
Fix Version/s: 0.13.0
   (was: 0.12.1)

> Make sure all DeltaStreamer Sources are Closeable
> -
>
> Key: HUDI-4624
> URL: https://issues.apache.org/jira/browse/HUDI-4624
> Project: Apache Hudi
>  Issue Type: Bug
>  Components: deltastreamer
>Reporter: Alexey Kudinkin
>Assignee: sivabalan narayanan
>Priority: Blocker
> Fix For: 0.13.0
>
>
> Currently, many of the sources acquire resources (creating 
> Kafka/Pulsar/SQS/etc clients) but are not releasing them properly, since they 
> don't implement `Closeable` interface proeprly.
> We need to rectify that by making sure that any DS Source that acquires some 
> resource, releases it properly in `close`.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Updated] (HUDI-4586) Address S3 timeouts in Bloom Index with metadata table

2022-10-01 Thread Zhaojing Yu (Jira)


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

Zhaojing Yu updated HUDI-4586:
--
Fix Version/s: 0.13.0
   (was: 0.12.1)

> Address S3 timeouts in Bloom Index with metadata table
> --
>
> Key: HUDI-4586
> URL: https://issues.apache.org/jira/browse/HUDI-4586
> Project: Apache Hudi
>  Issue Type: Improvement
>Reporter: Ethan Guo
>Assignee: Ethan Guo
>Priority: Blocker
>  Labels: pull-request-available
> Fix For: 0.13.0
>
> Attachments: Screen Shot 2022-08-15 at 17.39.01.png
>
>
> For partitioned table, there are significant number of S3 requests timeout 
> causing the upserts to fail when using Bloom Index with metadata table.
> {code:java}
> Load meta index key ranges for file slices: hudi
> collect at HoodieSparkEngineContext.java:137+details
> org.apache.spark.api.java.AbstractJavaRDDLike.collect(JavaRDDLike.scala:45)
> org.apache.hudi.client.common.HoodieSparkEngineContext.flatMap(HoodieSparkEngineContext.java:137)
> org.apache.hudi.index.bloom.HoodieBloomIndex.loadColumnRangesFromMetaIndex(HoodieBloomIndex.java:213)
> org.apache.hudi.index.bloom.HoodieBloomIndex.getBloomIndexFileInfoForPartitions(HoodieBloomIndex.java:145)
> org.apache.hudi.index.bloom.HoodieBloomIndex.lookupIndex(HoodieBloomIndex.java:123)
> org.apache.hudi.index.bloom.HoodieBloomIndex.tagLocation(HoodieBloomIndex.java:89)
> org.apache.hudi.table.action.commit.HoodieWriteHelper.tag(HoodieWriteHelper.java:49)
> org.apache.hudi.table.action.commit.HoodieWriteHelper.tag(HoodieWriteHelper.java:32)
> org.apache.hudi.table.action.commit.BaseWriteHelper.write(BaseWriteHelper.java:53)
> org.apache.hudi.table.action.commit.SparkUpsertCommitActionExecutor.execute(SparkUpsertCommitActionExecutor.java:45)
> org.apache.hudi.table.HoodieSparkCopyOnWriteTable.upsert(HoodieSparkCopyOnWriteTable.java:113)
> org.apache.hudi.table.HoodieSparkCopyOnWriteTable.upsert(HoodieSparkCopyOnWriteTable.java:97)
> org.apache.hudi.client.SparkRDDWriteClient.upsert(SparkRDDWriteClient.java:155)
> org.apache.hudi.DataSourceUtils.doWriteOperation(DataSourceUtils.java:206)
> org.apache.hudi.HoodieSparkSqlWriter$.write(HoodieSparkSqlWriter.scala:329)
> org.apache.hudi.DefaultSource.createRelation(DefaultSource.scala:183)
> org.apache.spark.sql.execution.datasources.SaveIntoDataSourceCommand.run(SaveIntoDataSourceCommand.scala:45)
> org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult$lzycompute(commands.scala:75)
> org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult(commands.scala:73)
> org.apache.spark.sql.execution.command.ExecutedCommandExec.executeCollect(commands.scala:84)
>  {code}
> {code:java}
> org.apache.hudi.exception.HoodieException: Exception when reading log file 
>     at 
> org.apache.hudi.common.table.log.AbstractHoodieLogRecordReader.scanInternal(AbstractHoodieLogRecordReader.java:352)
>     at 
> org.apache.hudi.common.table.log.AbstractHoodieLogRecordReader.scan(AbstractHoodieLogRecordReader.java:196)
>     at 
> org.apache.hudi.metadata.HoodieMetadataMergedLogRecordReader.getRecordsByKeys(HoodieMetadataMergedLogRecordReader.java:124)
>     at 
> org.apache.hudi.metadata.HoodieBackedTableMetadata.readLogRecords(HoodieBackedTableMetadata.java:266)
>     at 
> org.apache.hudi.metadata.HoodieBackedTableMetadata.lambda$getRecordsByKeys$1(HoodieBackedTableMetadata.java:222)
>     at java.util.HashMap.forEach(HashMap.java:1290)
>     at 
> org.apache.hudi.metadata.HoodieBackedTableMetadata.getRecordsByKeys(HoodieBackedTableMetadata.java:209)
>     at 
> org.apache.hudi.metadata.BaseTableMetadata.getColumnStats(BaseTableMetadata.java:253)
>     at 
> org.apache.hudi.index.bloom.HoodieBloomIndex.lambda$loadColumnRangesFromMetaIndex$cc8e7ca2$1(HoodieBloomIndex.java:224)
>     at 
> org.apache.hudi.client.common.HoodieSparkEngineContext.lambda$flatMap$7d470b86$1(HoodieSparkEngineContext.java:137)
>     at 
> org.apache.spark.api.java.JavaRDDLike.$anonfun$flatMap$1(JavaRDDLike.scala:125)
>     at scala.collection.Iterator$$anon$11.nextCur(Iterator.scala:486)
>     at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:492)
>     at scala.collection.Iterator.foreach(Iterator.scala:943)
>     at scala.collection.Iterator.foreach$(Iterator.scala:943)
>     at scala.collection.AbstractIterator.foreach(Iterator.scala:1431)
>     at scala.collection.generic.Growable.$plus$plus$eq(Growable.scala:62)
>     at scala.collection.generic.Growable.$plus$plus$eq$(Growable.scala:53)
>     at 
> scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:105)
>     at 
> scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:49)
>     at scala.collection.TraversableOnce.to(TraversableOnce.scala:366)
>    

[jira] [Updated] (HUDI-4602) Implement getScheme for PrestoS3FileSystem

2022-10-01 Thread Zhaojing Yu (Jira)


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

Zhaojing Yu updated HUDI-4602:
--
Fix Version/s: 0.13.0
   (was: 0.12.1)

> Implement getScheme for PrestoS3FileSystem
> --
>
> Key: HUDI-4602
> URL: https://issues.apache.org/jira/browse/HUDI-4602
> Project: Apache Hudi
>  Issue Type: Task
>Reporter: Sagar Sumit
>Priority: Critical
> Fix For: 0.13.0
>
>
> In absence of this method, we get the below exception, unless we are running 
> on EMR (where EmrFileSystem gets picked as s3 file system type and that 
> implements this method already).
> {code:java}
> java.lang.UnsupportedOperationException: Not implemented by the 
> PrestoS3FileSystem FileSystem implementation
>   at org.apache.hadoop.fs.FileSystem.getScheme(FileSystem.java:219)
>   at 
> org.apache.hadoop.fs.HadoopExtendedFileSystem.getScheme(HadoopExtendedFileSystem.java:71)
>   at org.apache.hudi.common.fs.FSUtils.isGCSFileSystem(FSUtils.java:592)
>   at 
> org.apache.hudi.common.table.log.HoodieLogFileReader.getFSDataInputStream(HoodieLogFileReader.java:119)
>   at 
> org.apache.hudi.common.table.log.HoodieLogFileReader.(HoodieLogFileReader.java:95)
>   at 
> org.apache.hudi.common.table.log.HoodieLogFileReader.(HoodieLogFileReader.java:86)
>   at 
> org.apache.hudi.common.table.log.HoodieLogFormat.newReader(HoodieLogFormat.java:282)
>   at 
> org.apache.hudi.common.table.log.LogReaderUtils.readSchemaFromLogFileInReverse(LogReaderUtils.java:49)
>   at 
> org.apache.hudi.common.table.log.LogReaderUtils.readLatestSchemaFromLogFiles(LogReaderUtils.java:77)
>   at 
> org.apache.hudi.hadoop.realtime.AbstractRealtimeRecordReader.init(AbstractRealtimeRecordReader.java:85)
>   at 
> org.apache.hudi.hadoop.realtime.AbstractRealtimeRecordReader.(AbstractRealtimeRecordReader.java:67)
>   at 
> org.apache.hudi.hadoop.realtime.RealtimeCompactedRecordReader.(RealtimeCompactedRecordReader.java:62)
>   at 
> org.apache.hudi.hadoop.realtime.HoodieRealtimeRecordReader.constructRecordReader(HoodieRealtimeRecordReader.java:70)
>   at 
> org.apache.hudi.hadoop.realtime.HoodieRealtimeRecordReader.(HoodieRealtimeRecordReader.java:47)
>   at 
> org.apache.hudi.hadoop.realtime.HoodieParquetRealtimeInputFormat.getRecordReader(HoodieParquetRealtimeInputFormat.java:323)
>   at 
> com.facebook.presto.hive.HiveUtil.createRecordReader(HiveUtil.java:272)
>   at 
> com.facebook.presto.hive.GenericHiveRecordCursorProvider.lambda$createRecordCursor$0(GenericHiveRecordCursorProvider.java:74)
>   at 
> com.facebook.presto.hive.authentication.NoHdfsAuthentication.doAs(NoHdfsAuthentication.java:23)
>   at 
> com.facebook.presto.hive.HdfsEnvironment.doAs(HdfsEnvironment.java:81)
>   at 
> com.facebook.presto.hive.GenericHiveRecordCursorProvider.createRecordCursor(GenericHiveRecordCursorProvider.java:73)
>   at 
> com.facebook.presto.hive.HivePageSourceProvider.getPageSourceFromCursorProvider(HivePageSourceProvider.java:573)
>   at 
> com.facebook.presto.hive.HivePageSourceProvider.createHivePageSource(HivePageSourceProvider.java:414)
>   at 
> com.facebook.presto.hive.HivePageSourceProvider.createPageSource(HivePageSourceProvider.java:186)
>   at 
> com.facebook.presto.spi.connector.classloader.ClassLoaderSafeConnectorPageSourceProvider.createPageSource(ClassLoaderSafeConnectorPageSourceProvider.java:63)
>   at 
> com.facebook.presto.split.PageSourceManager.createPageSource(PageSourceManager.java:80)
>   at 
> com.facebook.presto.operator.ScanFilterAndProjectOperator.getOutput(ScanFilterAndProjectOperator.java:235)
>   at com.facebook.presto.operator.Driver.processInternal(Driver.java:424)
>   at 
> com.facebook.presto.operator.Driver.lambda$processFor$9(Driver.java:307)
>   at com.facebook.presto.operator.Driver.tryWithLock(Driver.java:728)
>   at com.facebook.presto.operator.Driver.processFor(Driver.java:300)
>   at 
> com.facebook.presto.execution.SqlTaskExecution$DriverSplitRunner.processFor(SqlTaskExecution.java:1079)
>   at 
> com.facebook.presto.execution.executor.PrioritizedSplitRunner.process(PrioritizedSplitRunner.java:162)
>   at 
> com.facebook.presto.execution.executor.TaskExecutor$TaskRunner.run(TaskExecutor.java:599)
>   at 
> com.facebook.presto.$gen.Presto_0_270_AHN_0_620220811_002110_1.run(Unknown
>  Source)
>   at 
> java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
>   at 
> java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
>   at java.base/java.lang.Thread.run(Thread.java:829) {code}



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Updated] (HUDI-4674) change the default value of inputFormat for the MOR table

2022-10-01 Thread Zhaojing Yu (Jira)


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

Zhaojing Yu updated HUDI-4674:
--
Fix Version/s: 0.13.0
   (was: 0.12.1)

> change the default value of inputFormat for the MOR table
> -
>
> Key: HUDI-4674
> URL: https://issues.apache.org/jira/browse/HUDI-4674
> Project: Apache Hudi
>  Issue Type: Improvement
>Reporter: linfey.nie
>Assignee: linfey.nie
>Priority: Major
>  Labels: hudi-on-call, pull-request-available
> Fix For: 0.13.0
>
>
> When we build a mor table, for example with Sparksql,the default value of 
> inputFormat is HoodieParquetRealtimeInputFormat.but when use hive sync 
> metadata and skip the _ro suffix for Read,The inputFormat of the original 
> table name should be HoodieParquetInputFormat,but now is not.I think we 
> should change the default value of inputFormat,just like cow table.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Updated] (HUDI-4631) Enhance retries for failed writes w/ write conflicts in a multi writer scenarios

2022-10-01 Thread Zhaojing Yu (Jira)


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

Zhaojing Yu updated HUDI-4631:
--
Fix Version/s: 0.13.0
   (was: 0.12.1)

> Enhance retries for failed writes w/ write conflicts in a multi writer 
> scenarios
> 
>
> Key: HUDI-4631
> URL: https://issues.apache.org/jira/browse/HUDI-4631
> Project: Apache Hudi
>  Issue Type: Improvement
>  Components: multi-writer
>Reporter: sivabalan narayanan
>Priority: Blocker
> Fix For: 0.13.0
>
>
> lets say there are two writers from t0 to t5. so hudi fails w2 and succeeds 
> w1. and user restarts w2 and for next 5 mins, lets say there are no other 
> overlapping writers. So the same write from w2 will now succeed. so, whenever 
> there is a write conflict and pipeline fails, all user needs to do is, just 
> restart the pipeline or retry to ingest the same batch.
>  
> Ask: can we add retries within hudi during such failures. Anyways, in most 
> cases, users just restart the pipeline in such cases. 
>  



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Updated] (HUDI-4688) Decouple lazy cleaning of failed writes from clean action in multi-writer

2022-10-01 Thread Zhaojing Yu (Jira)


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

Zhaojing Yu updated HUDI-4688:
--
Fix Version/s: 0.13.0
   (was: 0.12.1)

> Decouple lazy cleaning of failed writes from clean action in multi-writer
> -
>
> Key: HUDI-4688
> URL: https://issues.apache.org/jira/browse/HUDI-4688
> Project: Apache Hudi
>  Issue Type: Improvement
>Reporter: Ethan Guo
>Priority: Major
> Fix For: 0.13.0
>
>




--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Updated] (HUDI-4704) bulk insert overwrite table will delete the table and then recreate a table

2022-10-01 Thread Zhaojing Yu (Jira)


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

Zhaojing Yu updated HUDI-4704:
--
Fix Version/s: 0.13.0
   (was: 0.12.1)

> bulk insert overwrite table will delete the table and then recreate a table
> ---
>
> Key: HUDI-4704
> URL: https://issues.apache.org/jira/browse/HUDI-4704
> Project: Apache Hudi
>  Issue Type: Bug
>  Components: spark-sql, writer-core
>Affects Versions: 0.12.0
>Reporter: zouxxyy
>Assignee: Raymond Xu
>Priority: Major
> Fix For: 0.13.0
>
>
> When hoodie.sql.bulk.insert.enable is enabled, executing insert overwrite 
> will delete the table and then recreate a table, so time travel cannot be 
> performed.
>  
> {code:java}
> create table hudi_cow_test_tbl (
>   id bigint,
>   name string,
>   ts bigint,
>   dt string,
>   hh string
> ) using hudi
> tblproperties (
>   type = 'cow',
>   primaryKey = 'id',
>   preCombineField = 'ts',
>   'hoodie.sql.insert.mode' = 'non-strict',
>   'hoodie.sql.bulk.insert.enable' = 'true'
> );
> insert into hudi_cow_test_tbl select 1, 'a1', 1001, '2021-12-09', '11';
> insert overwrite hudi_cow_test_tbl select 3, 'a3', 1001, '2021-12-09', '11';
> {code}



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[GitHub] [hudi] hudi-bot commented on pull request #6848: [HUDI-4769] Option read.streaming.skip_compaction skips delta commit

2022-10-01 Thread GitBox


hudi-bot commented on PR #6848:
URL: https://github.com/apache/hudi/pull/6848#issuecomment-1264346736

   
   ## CI report:
   
   * 6318c29ab4e0415d365db365aac92f1f562a4b68 Azure: 
[PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11954)
 
   * fcfc1baee8aa5394c4472135f6ceb0d280cf8c16 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



  1   2   >