[GitHub] [hudi] zhangyue19921010 commented on a diff in pull request #5416: [HUDI-3963] Use Lock-Free Message Queue Disruptor Improving Hoodie Writing Efficiency

2022-10-06 Thread GitBox


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


##
hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorExecutor.java:
##
@@ -0,0 +1,168 @@
+/*
+ * 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;
+
+import com.lmax.disruptor.dsl.Disruptor;
+import org.apache.hudi.common.util.CustomizedThreadFactory;
+import org.apache.hudi.common.util.Functions;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.exception.HoodieException;
+
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+public class DisruptorExecutor extends HoodieExecutor {
+
+  private static final Logger LOG = 
LogManager.getLogger(DisruptorExecutor.class);
+
+  // Executor service used for launching write thread.
+  private final ExecutorService producerExecutorService;
+  // Executor service used for launching read thread.
+  private final ExecutorService consumerExecutorService;
+  // Used for buffering records which is controlled by 
HoodieWriteConfig#WRITE_BUFFER_LIMIT_BYTES.
+  private final DisruptorMessageQueue queue;
+  // Producers
+  private final List> producers;
+  // Consumer
+  private final Option> consumer;
+  // pre-execute function to implement environment specific behavior before 
executors (producers/consumer) run
+  private final Runnable preExecuteRunnable;
+
+  public DisruptorExecutor(final int bufferSize, final Iterator inputItr,
+   BoundedInMemoryQueueConsumer consumer, 
Function transformFunction, String waitStrategy, Runnable 
preExecuteRunnable) {
+this(bufferSize, new IteratorBasedDisruptorProducer<>(inputItr), 
Option.of(consumer), transformFunction, waitStrategy, preExecuteRunnable);
+  }
+
+  public DisruptorExecutor(final int bufferSize, DisruptorBasedProducer 
producer,
+   Option> 
consumer, final Function transformFunction) {
+this(bufferSize, producer, consumer, transformFunction, 
WaitStrategyFactory.DEFAULT_STRATEGY, Functions.noop());
+  }
+
+  public DisruptorExecutor(final int bufferSize, DisruptorBasedProducer 
producer,
+   Option> 
consumer, final Function transformFunction, String waitStrategy, Runnable 
preExecuteRunnable) {
+this(bufferSize, Collections.singletonList(producer), consumer, 
transformFunction, waitStrategy, preExecuteRunnable);
+  }
+
+  public DisruptorExecutor(final int bufferSize, 
List> producers,
+   Option> 
consumer, final Function transformFunction,
+   final String waitStrategy, Runnable 
preExecuteRunnable) {
+this.producers = producers;
+this.consumer = consumer;
+this.preExecuteRunnable = preExecuteRunnable;
+// Ensure fixed thread for each producer thread
+this.producerExecutorService = 
Executors.newFixedThreadPool(producers.size(), new 
CustomizedThreadFactory("producer"));
+// Ensure single thread for consumer
+this.consumerExecutorService = Executors.newSingleThreadExecutor(new 
CustomizedThreadFactory("consumer"));

Review Comment:
   Since we now have HoodieExecutorBase it holds common elements 
producerExecutorService, consumerExecutorService, producers and consumer and 
share the same logic for creating consumerExecutorService here.



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

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

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



[GitHub] [hudi] zhangyue19921010 commented on a diff in pull request #5416: [HUDI-3963] Use Lock-Free Message Queue Disruptor Improving Hoodie Writing Efficiency

2022-10-06 Thread GitBox


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


##
hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageQueue.java:
##
@@ -0,0 +1,80 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *  http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.common.util.queue;
+
+import com.lmax.disruptor.EventTranslator;
+import com.lmax.disruptor.RingBuffer;
+import com.lmax.disruptor.WaitStrategy;
+import com.lmax.disruptor.dsl.Disruptor;
+import com.lmax.disruptor.dsl.ProducerType;
+
+import java.util.function.Function;
+
+public class DisruptorMessageQueue extends HoodieMessageQueue {
+
+  private final Disruptor> queue;
+  private final Function transformFunction;
+  private RingBuffer> ringBuffer;

Review Comment:
   changed!



##
hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageQueue.java:
##
@@ -0,0 +1,80 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *  http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.common.util.queue;
+
+import com.lmax.disruptor.EventTranslator;
+import com.lmax.disruptor.RingBuffer;
+import com.lmax.disruptor.WaitStrategy;
+import com.lmax.disruptor.dsl.Disruptor;
+import com.lmax.disruptor.dsl.ProducerType;
+
+import java.util.function.Function;
+
+public class DisruptorMessageQueue extends HoodieMessageQueue {
+
+  private final Disruptor> queue;
+  private final Function transformFunction;
+  private RingBuffer> ringBuffer;
+
+  public DisruptorMessageQueue(int bufferSize, Function 
transformFunction, String waitStrategyName, int producerNumber, Runnable 
preExecuteRunnable) {
+WaitStrategy waitStrategy = WaitStrategyFactory.build(waitStrategyName);
+HoodieDaemonThreadFactory threadFactory = new 
HoodieDaemonThreadFactory(preExecuteRunnable);
+
+if (producerNumber > 1) {
+  this.queue = new Disruptor<>(HoodieDisruptorEvent::new, bufferSize, 
threadFactory, ProducerType.MULTI, waitStrategy);
+} else {
+  this.queue = new Disruptor<>(HoodieDisruptorEvent::new, bufferSize, 
threadFactory, ProducerType.SINGLE, waitStrategy);
+}
+
+this.ringBuffer = queue.getRingBuffer();
+this.transformFunction = transformFunction;
+  }
+
+  @Override
+  public long size() {
+return queue.getBufferSize();
+  }
+
+  @Override
+  public void insertRecord(I value) throws Exception {
+O applied = transformFunction.apply(value);
+
+EventTranslator> translator = new 
EventTranslator>() {
+  @Override
+  public void translateTo(HoodieDisruptorEvent event, long sequence) {
+event.set(applied);
+  }
+};
+
+queue.getRingBuffer().publishEvent(translator);
+  }
+
+  @Override
+  public void close() {
+queue.shutdown();
+  }
+
+  public Disruptor> getInnerQueue() {

Review Comment:
   changed!



-- 
This is an automated message from the 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] zhangyue19921010 commented on a diff in pull request #5416: [HUDI-3963] Use Lock-Free Message Queue Disruptor Improving Hoodie Writing Efficiency

2022-10-06 Thread GitBox


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


##
hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorExecutor.java:
##
@@ -0,0 +1,163 @@
+/*
+ * 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;
+
+import com.lmax.disruptor.dsl.Disruptor;
+import org.apache.hudi.common.util.CustomizedThreadFactory;
+import org.apache.hudi.common.util.Functions;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.exception.HoodieException;
+
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+public class DisruptorExecutor extends HoodieExecutor {
+
+  private static final Logger LOG = 
LogManager.getLogger(DisruptorExecutor.class);
+
+  // Executor service used for launching write thread.
+  private final ExecutorService producerExecutorService;
+  // Executor service used for launching read thread.
+  private final ExecutorService consumerExecutorService;
+  // Used for buffering records which is controlled by 
HoodieWriteConfig#WRITE_BUFFER_LIMIT_BYTES.
+  private final DisruptorMessageQueue queue;
+  // Producers
+  private final List> producers;
+  // Consumer
+  private final Option> consumer;
+  // pre-execute function to implement environment specific behavior before 
executors (producers/consumer) run
+  private final Runnable preExecuteRunnable;
+
+  public DisruptorExecutor(final int bufferSize, final Iterator inputItr,
+   BoundedInMemoryQueueConsumer consumer, 
Function transformFunction, String waitStrategy, Runnable 
preExecuteRunnable) {
+this(bufferSize, new IteratorBasedDisruptorProducer<>(inputItr), 
Option.of(consumer), transformFunction, waitStrategy, preExecuteRunnable);
+  }
+
+  public DisruptorExecutor(final int bufferSize, DisruptorBasedProducer 
producer,
+   Option> 
consumer, final Function transformFunction) {
+this(bufferSize, producer, consumer, transformFunction, 
WaitStrategyFactory.DEFAULT_STRATEGY, Functions.noop());
+  }
+
+  public DisruptorExecutor(final int bufferSize, DisruptorBasedProducer 
producer,
+   Option> 
consumer, final Function transformFunction, String waitStrategy, Runnable 
preExecuteRunnable) {
+this(bufferSize, Collections.singletonList(producer), consumer, 
transformFunction, waitStrategy, preExecuteRunnable);
+  }
+
+  public DisruptorExecutor(final int bufferSize, 
List> producers,
+   Option> 
consumer, final Function transformFunction,
+   final String waitStrategy, Runnable 
preExecuteRunnable) {
+this.producers = producers;
+this.consumer = consumer;
+this.preExecuteRunnable = preExecuteRunnable;
+// Ensure fixed thread for each producer thread
+this.producerExecutorService = 
Executors.newFixedThreadPool(producers.size(), new 
CustomizedThreadFactory("producer"));
+// Ensure single thread for consumer
+this.consumerExecutorService = Executors.newSingleThreadExecutor(new 
CustomizedThreadFactory("consumer"));
+this.queue = new DisruptorMessageQueue<>(bufferSize, transformFunction, 
waitStrategy, producers.size(), preExecuteRunnable);
+  }
+
+  /**
+   * Start all Producers.
+   */
+  public ExecutorCompletionService startProducers() {
+final ExecutorCompletionService completionService =
+new ExecutorCompletionService(producerExecutorService);
+producers.stream().map(producer -> {
+  return completionService.submit(() -> {
+try {
+  preExecuteRunnable.run();
+
+  DisruptorPublisher publisher = new DisruptorPublisher<>(producer, 
queue);
+  publisher.startProduce();
+
+} catch (Throwable e) {
+  LOG.error("error producing records", e);
+  throw 

[GitHub] [hudi] zhangyue19921010 commented on a diff in pull request #5416: [HUDI-3963] Use Lock-Free Message Queue Disruptor Improving Hoodie Writing Efficiency

2022-10-06 Thread GitBox


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


##
hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorExecutor.java:
##
@@ -0,0 +1,168 @@
+/*
+ * 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;
+
+import com.lmax.disruptor.dsl.Disruptor;
+import org.apache.hudi.common.util.CustomizedThreadFactory;
+import org.apache.hudi.common.util.Functions;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.exception.HoodieException;
+
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+public class DisruptorExecutor extends HoodieExecutor {
+
+  private static final Logger LOG = 
LogManager.getLogger(DisruptorExecutor.class);
+
+  // Executor service used for launching write thread.
+  private final ExecutorService producerExecutorService;
+  // Executor service used for launching read thread.
+  private final ExecutorService consumerExecutorService;
+  // Used for buffering records which is controlled by 
HoodieWriteConfig#WRITE_BUFFER_LIMIT_BYTES.
+  private final DisruptorMessageQueue queue;
+  // Producers
+  private final List> producers;
+  // Consumer
+  private final Option> consumer;
+  // pre-execute function to implement environment specific behavior before 
executors (producers/consumer) run
+  private final Runnable preExecuteRunnable;
+
+  public DisruptorExecutor(final int bufferSize, final Iterator inputItr,
+   BoundedInMemoryQueueConsumer consumer, 
Function transformFunction, String waitStrategy, Runnable 
preExecuteRunnable) {
+this(bufferSize, new IteratorBasedDisruptorProducer<>(inputItr), 
Option.of(consumer), transformFunction, waitStrategy, preExecuteRunnable);
+  }
+
+  public DisruptorExecutor(final int bufferSize, DisruptorBasedProducer 
producer,
+   Option> 
consumer, final Function transformFunction) {
+this(bufferSize, producer, consumer, transformFunction, 
WaitStrategyFactory.DEFAULT_STRATEGY, Functions.noop());
+  }
+
+  public DisruptorExecutor(final int bufferSize, DisruptorBasedProducer 
producer,
+   Option> 
consumer, final Function transformFunction, String waitStrategy, Runnable 
preExecuteRunnable) {
+this(bufferSize, Collections.singletonList(producer), consumer, 
transformFunction, waitStrategy, preExecuteRunnable);
+  }
+
+  public DisruptorExecutor(final int bufferSize, 
List> producers,
+   Option> 
consumer, final Function transformFunction,
+   final String waitStrategy, Runnable 
preExecuteRunnable) {
+this.producers = producers;
+this.consumer = consumer;
+this.preExecuteRunnable = preExecuteRunnable;
+// Ensure fixed thread for each producer thread
+this.producerExecutorService = 
Executors.newFixedThreadPool(producers.size(), new 
CustomizedThreadFactory("producer"));
+// Ensure single thread for consumer
+this.consumerExecutorService = Executors.newSingleThreadExecutor(new 
CustomizedThreadFactory("consumer"));
+this.queue = new DisruptorMessageQueue<>(bufferSize, transformFunction, 
waitStrategy, producers.size(), preExecuteRunnable);
+  }
+
+  /**
+   * Start all Producers.
+   */
+  public ExecutorCompletionService startProducers() {
+final ExecutorCompletionService completionService =
+new ExecutorCompletionService(producerExecutorService);
+producers.stream().map(producer -> {
+  return completionService.submit(() -> {
+try {
+  preExecuteRunnable.run();
+
+  DisruptorPublisher publisher = new DisruptorPublisher<>(producer, 
queue);
+  publisher.startProduce();
+
+} catch (Throwable e) {
+  LOG.error("error producing records", e);
+  throw 

[GitHub] [hudi] zhangyue19921010 commented on a diff in pull request #5416: [HUDI-3963] Use Lock-Free Message Queue Disruptor Improving Hoodie Writing Efficiency

2022-10-06 Thread GitBox


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


##
hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorExecutor.java:
##
@@ -0,0 +1,168 @@
+/*
+ * 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;
+
+import com.lmax.disruptor.dsl.Disruptor;
+import org.apache.hudi.common.util.CustomizedThreadFactory;
+import org.apache.hudi.common.util.Functions;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.exception.HoodieException;
+
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+public class DisruptorExecutor extends HoodieExecutor {
+
+  private static final Logger LOG = 
LogManager.getLogger(DisruptorExecutor.class);
+
+  // Executor service used for launching write thread.
+  private final ExecutorService producerExecutorService;

Review Comment:
   nice idea. Changed!



-- 
This is an automated message from the 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] zhangyue19921010 commented on a diff in pull request #5416: [HUDI-3963] Use Lock-Free Message Queue Disruptor Improving Hoodie Writing Efficiency

2022-10-06 Thread GitBox


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


##
hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorBasedProducer.java:
##
@@ -0,0 +1,29 @@
+/*
+ * 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 abstract class DisruptorBasedProducer implements HoodieProducer {

Review Comment:
   Nice catch! totally agree!
   
   abstract HoodieConsumer and HoodieProducer. They modify HoodieMessageQueue 
and don't care what the specific implementation of HoodieMessageQueue is 



-- 
This is an automated message from the 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] zhangyue19921010 commented on a diff in pull request #5416: [HUDI-3963] Use Lock-Free Message Queue Disruptor Improving Hoodie Writing Efficiency

2022-10-06 Thread GitBox


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


##
hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieDaemonThreadFactory.java:
##
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *  http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.common.util.queue;
+
+import org.jetbrains.annotations.NotNull;

Review Comment:
   No need for that actually, just removed.



##
hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieConsumer.java:
##
@@ -0,0 +1,37 @@
+/*
+ * 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 abstract class HoodieConsumer {
+
+  /**
+   * Consumer One record.
+   */
+  protected abstract void consumeOneRecord(I record);
+
+  /**
+   * Notifies implementation that we have exhausted consuming records from 
queue.
+   */
+  protected abstract void finish();
+
+  /**
+   * Return result of consuming records so far.
+   */
+  protected abstract O getResult();

Review Comment:
   No need for that actually, just removed.



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

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

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



[GitHub] [hudi] zhangyue19921010 commented on a diff in pull request #5416: [HUDI-3963] Use Lock-Free Message Queue Disruptor Improving Hoodie Writing Efficiency

2022-10-06 Thread GitBox


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


##
hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieDaemonThreadFactory.java:
##
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *  http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.common.util.queue;
+
+import org.jetbrains.annotations.NotNull;
+import java.util.concurrent.ThreadFactory;
+
+public class HoodieDaemonThreadFactory implements ThreadFactory {
+
+  private Runnable preExecuteRunnable;
+
+  public HoodieDaemonThreadFactory(Runnable preExecuteRunnable) {

Review Comment:
   Changed!



-- 
This is an automated message from the 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] zhangyue19921010 commented on a diff in pull request #5416: [HUDI-3963] Use Lock-Free Message Queue Disruptor Improving Hoodie Writing Efficiency

2022-10-06 Thread GitBox


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


##
hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieDaemonThreadFactory.java:
##
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *  http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.common.util.queue;
+
+import org.jetbrains.annotations.NotNull;
+import java.util.concurrent.ThreadFactory;
+
+public class HoodieDaemonThreadFactory implements ThreadFactory {
+
+  private Runnable preExecuteRunnable;
+
+  public HoodieDaemonThreadFactory(Runnable preExecuteRunnable) {
+this.preExecuteRunnable = preExecuteRunnable;

Review Comment:
   ```
 public Runnable getPreExecuteRunnable() {
   final TaskContext taskContext = TaskContext.get();
   return () -> TaskContext$.MODULE$.setTaskContext(taskContext);
 }
   ```
   
   We have to set up spark task context in each thread. So that we can get task 
info.



-- 
This is an automated message from the 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] zhangyue19921010 commented on a diff in pull request #5416: [HUDI-3963] Use Lock-Free Message Queue Disruptor Improving Hoodie Writing Efficiency

2022-10-06 Thread GitBox


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


##
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:
   Yeap. as nsivabalan said, we need to warp record/message as an event that 
can works in disruptor world.



-- 
This is an automated message from the 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] zhangyue19921010 commented on a diff in pull request #5416: [HUDI-3963] Use Lock-Free Message Queue Disruptor Improving Hoodie Writing Efficiency

2022-10-06 Thread GitBox


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


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

Review Comment:
   > Same comments as above (please ditto for all interfaces)
   
   Changed all.
   
   > This API seems incomplete: we do have methods to insert into the queue, 
but we don't have a method to take from the queue
   
   Now there are 4 func in interface HoodieMessageQueue
   ```
   public interface HoodieMessageQueue {
   
 /**
  * Get the size of inner message queue.
  */
 long size();
   
 /**
  * Insert a record into inner message queue.
  */
 void insertRecord(I t) throws Exception;
   
 /**
  * Read records from inner message queue.
  */
 Option readNextRecord();
   
 void close();
   }
   ```



##
hudi-common/src/main/java/org/apache/hudi/common/util/queue/WaitStrategyFactory.java:
##
@@ -0,0 +1,53 @@
+/*
+ * 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;
+
+import static 
org.apache.hudi.common.util.queue.DisruptorWaitStrategyType.BLOCKINGWAITSTRATEGY;
+
+import com.lmax.disruptor.BlockingWaitStrategy;
+import com.lmax.disruptor.BusySpinWaitStrategy;
+import com.lmax.disruptor.SleepingWaitStrategy;
+import com.lmax.disruptor.WaitStrategy;
+import com.lmax.disruptor.YieldingWaitStrategy;
+import org.apache.hudi.exception.HoodieException;
+
+public class WaitStrategyFactory {
+
+  public static final String DEFAULT_STRATEGY = BLOCKINGWAITSTRATEGY.name();
+
+  /**
+   * Build WaitStrategy for disruptor
+   */
+  public static WaitStrategy build(String name) {
+
+DisruptorWaitStrategyType strategyType = 
DisruptorWaitStrategyType.valueOf(name.toUpperCase());
+switch (strategyType) {
+  case BLOCKINGWAITSTRATEGY:

Review Comment:
   changed!



-- 
This is an automated message from the 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-06 Thread GitBox


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

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

2022-10-06 Thread GitBox


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

   
   ## CI report:
   
   * b838e1f406902c9bdfb5e84d53ef5a5effd0765b UNKNOWN
   * 6114ee2aa59f087e5ef0b1b53979eec143b33f5e UNKNOWN
   * 92760dbf5a047fe1f9941fa4b36c944eb3bec5c7 UNKNOWN
   * 3e44fb2397bdbfa9c63fab47e62f312451b91900 Azure: 
[SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12029)
 
   * 672fdc6665cb250678c0768f55c8086a3cbfc388 Azure: 
[PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12040)
 
   
   
   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-06 Thread GitBox


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

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

2022-10-06 Thread GitBox


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

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

2022-10-06 Thread GitBox


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


##
hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieExecutor.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;
+
+import java.util.concurrent.ExecutorCompletionService;
+
+public abstract class HoodieExecutor {

Review Comment:
   added.



-- 
This is an automated message from the 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] zhangyue19921010 commented on a diff in pull request #5416: [HUDI-3963] Use Lock-Free Message Queue Disruptor Improving Hoodie Writing Efficiency

2022-10-06 Thread GitBox


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


##
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java:
##
@@ -230,6 +240,16 @@ public class HoodieWriteConfig extends HoodieConfig {
   .defaultValue(String.valueOf(4 * 1024 * 1024))
   .withDocumentation("Size of in-memory buffer used for parallelizing 
network reads and lake storage writes.");
 
+  public static final ConfigProperty WRITE_BUFFER_SIZE = 
ConfigProperty
+  .key("hoodie.write.buffer.size")

Review Comment:
   Changed.



##
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java:
##
@@ -230,6 +240,16 @@ public class HoodieWriteConfig extends HoodieConfig {
   .defaultValue(String.valueOf(4 * 1024 * 1024))
   .withDocumentation("Size of in-memory buffer used for parallelizing 
network reads and lake storage writes.");
 
+  public static final ConfigProperty WRITE_BUFFER_SIZE = 
ConfigProperty
+  .key("hoodie.write.buffer.size")
+  .defaultValue(1024)
+  .withDocumentation("The size of the Disruptor Executor ring buffer, must 
be power of 2");
+
+  public static final ConfigProperty WRITE_WAIT_STRATEGY = 
ConfigProperty

Review Comment:
   Changed.



-- 
This is an automated message from the 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 #5416: [HUDI-3963] Use Lock-Free Message Queue Disruptor Improving Hoodie Writing Efficiency

2022-10-06 Thread GitBox


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

   
   ## CI report:
   
   * b838e1f406902c9bdfb5e84d53ef5a5effd0765b UNKNOWN
   * 6114ee2aa59f087e5ef0b1b53979eec143b33f5e UNKNOWN
   * 92760dbf5a047fe1f9941fa4b36c944eb3bec5c7 UNKNOWN
   * 3e44fb2397bdbfa9c63fab47e62f312451b91900 Azure: 
[SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12029)
 
   * 672fdc6665cb250678c0768f55c8086a3cbfc388 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 #6883: [HUDI-4992] Fixing invalid min/max record key stats in Parquet metadata

2022-10-06 Thread GitBox


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

   
   ## CI report:
   
   * c528116cecfd35edc4b6829cb091c4924dc1758f Azure: 
[SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12038)
 
   
   
   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-06 Thread GitBox


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

   
   ## CI report:
   
   * d99df9d68f351bd97aa8fee33cffc399f69efa9c Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12030)
 
   * 58c2a12352944f2d1794968ebcc676970bdb0bd0 Azure: 
[PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12039)
 
   
   
   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-06 Thread GitBox


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

   
   ## CI report:
   
   * d99df9d68f351bd97aa8fee33cffc399f69efa9c Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12030)
 
   * 58c2a12352944f2d1794968ebcc676970bdb0bd0 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] eshu commented on issue #6692: [SUPPORT] ClassCastException after migration to Hudi 0.12.0

2022-10-06 Thread GitBox


eshu commented on issue #6692:
URL: https://github.com/apache/hudi/issues/6692#issuecomment-1271026057

   hadoop-common have a dependency on avro, hbase-hadoop-compat, 
hadoop-mapreduce-client-core.
   
   I do not have dependencies on spark-avro, but probably Glue 3 contains it. 
And it is out of my control.


-- 
This is an automated message from the 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] eshu commented on issue #6692: [SUPPORT] ClassCastException after migration to Hudi 0.12.0

2022-10-06 Thread GitBox


eshu commented on issue #6692:
URL: https://github.com/apache/hudi/issues/6692#issuecomment-1271018152

   @xushiyan But I see spark-avro as a transitive dependency for 
hudi-aws-bundle. And hudi-aws-bundle has it as a direct and transitive 
dependency too.


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

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

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



[GitHub] [hudi] wzx140 closed pull request #6884: [Minor] fix comments

2022-10-06 Thread GitBox


wzx140 closed pull request #6884: [Minor] fix comments
URL: https://github.com/apache/hudi/pull/6884


-- 
This is an automated message from the 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] alexeykudinkin commented on a diff in pull request #6883: [HUDI-4992] Fixing invalid min/max record key stats in Parquet metadata

2022-10-06 Thread GitBox


alexeykudinkin commented on code in PR #6883:
URL: https://github.com/apache/hudi/pull/6883#discussion_r989616904


##
hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/storage/row/TestHoodieInternalRowParquetWriter.java:
##
@@ -68,44 +72,55 @@ public void tearDown() throws Exception {
 
   @ParameterizedTest
   @ValueSource(booleans = {true, false})
-  public void endToEndTest(boolean parquetWriteLegacyFormatEnabled) throws 
Exception {
+  public void testProperWriting(boolean parquetWriteLegacyFormatEnabled) 
throws Exception {
+// Generate inputs
+Dataset inputRows = SparkDatasetTestUtils.getRandomRows(sqlContext, 
100,
+HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, false);
+StructType schema = inputRows.schema();
+
+List rows = SparkDatasetTestUtils.toInternalRows(inputRows, 
SparkDatasetTestUtils.ENCODER);
+
 HoodieWriteConfig.Builder writeConfigBuilder =
 SparkDatasetTestUtils.getConfigBuilder(basePath, timelineServicePort);
-for (int i = 0; i < 5; i++) {
-  // init write support and parquet config
-  HoodieRowParquetWriteSupport writeSupport = 
getWriteSupport(writeConfigBuilder, hadoopConf, 
parquetWriteLegacyFormatEnabled);
-  HoodieWriteConfig cfg = writeConfigBuilder.build();
-  HoodieParquetConfig parquetConfig = new 
HoodieParquetConfig<>(writeSupport,
-  CompressionCodecName.SNAPPY, cfg.getParquetBlockSize(), 
cfg.getParquetPageSize(), cfg.getParquetMaxFileSize(),
-  writeSupport.getHadoopConf(), cfg.getParquetCompressionRatio(), 
cfg.parquetDictionaryEnabled());
-
-  // prepare path
-  String fileId = UUID.randomUUID().toString();
-  Path filePath = new Path(basePath + "/" + fileId);
-  String partitionPath = 
HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH;
-  metaClient.getFs().mkdirs(new Path(basePath));
-
-  // init writer
-  HoodieInternalRowParquetWriter writer = new 
HoodieInternalRowParquetWriter(filePath, parquetConfig);
-
-  // generate input
-  int size = 10 + RANDOM.nextInt(100);
-  // Generate inputs
-  Dataset inputRows = SparkDatasetTestUtils.getRandomRows(sqlContext, 
size, partitionPath, false);
-  List internalRows = 
SparkDatasetTestUtils.toInternalRows(inputRows, SparkDatasetTestUtils.ENCODER);
-
-  // issue writes
-  for (InternalRow internalRow : internalRows) {
-writer.write(internalRow);
-  }
 
-  // close the writer
-  writer.close();
+HoodieRowParquetWriteSupport writeSupport = 
getWriteSupport(writeConfigBuilder, hadoopConf, 
parquetWriteLegacyFormatEnabled);
+HoodieWriteConfig cfg = writeConfigBuilder.build();
+HoodieParquetConfig parquetConfig = new 
HoodieParquetConfig<>(writeSupport,
+CompressionCodecName.SNAPPY, cfg.getParquetBlockSize(), 
cfg.getParquetPageSize(), cfg.getParquetMaxFileSize(),

Review Comment:
   I'm carrying over this setup from the previous test and frankly i don't 
think the codec we use matter much in this test, but (practically) in every 
other test we actually test against the codec that is set in our config by 
default (since we're writing parquet in most of them)



-- 
This is an automated message from the 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] wzx140 opened a new pull request, #6884: [Minor] fix comments

2022-10-06 Thread GitBox


wzx140 opened a new pull request, #6884:
URL: https://github.com/apache/hudi/pull/6884

   ### Change Logs
   
   _Describe context and summary for this change. Highlight if any code was 
copied._
   
   ### Impact
   
   _Describe any public API or user-facing feature change or any performance 
impact._
   
   **Risk level: none | low | medium | high**
   
   _Choose one. If medium or high, explain what verification was done to 
mitigate the risks._
   
   ### Documentation Update
   
   _Describe any necessary documentation update if there is any new feature, 
config, or user-facing change_
   
   - _The config description must be updated if new configs are added or the 
default value of the configs are changed_
   - _Any new feature or user-facing change requires updating the Hudi website. 
Please create a Jira ticket, attach the
 ticket number here and follow the 
[instruction](https://hudi.apache.org/contribute/developer-setup#website) to 
make
 changes to the website._
   
   ### Contributor's checklist
   
   - [ ] Read through [contributor's 
guide](https://hudi.apache.org/contribute/how-to-contribute)
   - [ ] Change Logs and Impact were stated clearly
   - [ ] Adequate tests were added if applicable
   - [ ] CI passed
   


-- 
This is an automated message from the 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 #6883: [HUDI-4992] Fixing invalid min/max record key stats in Parquet metadata

2022-10-06 Thread GitBox


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


##
hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/storage/row/TestHoodieInternalRowParquetWriter.java:
##
@@ -68,44 +72,55 @@ public void tearDown() throws Exception {
 
   @ParameterizedTest
   @ValueSource(booleans = {true, false})
-  public void endToEndTest(boolean parquetWriteLegacyFormatEnabled) throws 
Exception {
+  public void testProperWriting(boolean parquetWriteLegacyFormatEnabled) 
throws Exception {
+// Generate inputs
+Dataset inputRows = SparkDatasetTestUtils.getRandomRows(sqlContext, 
100,
+HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, false);
+StructType schema = inputRows.schema();
+
+List rows = SparkDatasetTestUtils.toInternalRows(inputRows, 
SparkDatasetTestUtils.ENCODER);
+
 HoodieWriteConfig.Builder writeConfigBuilder =
 SparkDatasetTestUtils.getConfigBuilder(basePath, timelineServicePort);
-for (int i = 0; i < 5; i++) {
-  // init write support and parquet config
-  HoodieRowParquetWriteSupport writeSupport = 
getWriteSupport(writeConfigBuilder, hadoopConf, 
parquetWriteLegacyFormatEnabled);
-  HoodieWriteConfig cfg = writeConfigBuilder.build();
-  HoodieParquetConfig parquetConfig = new 
HoodieParquetConfig<>(writeSupport,
-  CompressionCodecName.SNAPPY, cfg.getParquetBlockSize(), 
cfg.getParquetPageSize(), cfg.getParquetMaxFileSize(),
-  writeSupport.getHadoopConf(), cfg.getParquetCompressionRatio(), 
cfg.parquetDictionaryEnabled());
-
-  // prepare path
-  String fileId = UUID.randomUUID().toString();
-  Path filePath = new Path(basePath + "/" + fileId);
-  String partitionPath = 
HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH;
-  metaClient.getFs().mkdirs(new Path(basePath));
-
-  // init writer
-  HoodieInternalRowParquetWriter writer = new 
HoodieInternalRowParquetWriter(filePath, parquetConfig);
-
-  // generate input
-  int size = 10 + RANDOM.nextInt(100);
-  // Generate inputs
-  Dataset inputRows = SparkDatasetTestUtils.getRandomRows(sqlContext, 
size, partitionPath, false);
-  List internalRows = 
SparkDatasetTestUtils.toInternalRows(inputRows, SparkDatasetTestUtils.ENCODER);
-
-  // issue writes
-  for (InternalRow internalRow : internalRows) {
-writer.write(internalRow);
-  }
 
-  // close the writer
-  writer.close();
+HoodieRowParquetWriteSupport writeSupport = 
getWriteSupport(writeConfigBuilder, hadoopConf, 
parquetWriteLegacyFormatEnabled);
+HoodieWriteConfig cfg = writeConfigBuilder.build();
+HoodieParquetConfig parquetConfig = new 
HoodieParquetConfig<>(writeSupport,
+CompressionCodecName.SNAPPY, cfg.getParquetBlockSize(), 
cfg.getParquetPageSize(), cfg.getParquetMaxFileSize(),

Review Comment:
   can we use default compression based on our config values. so that when we 
switch our default sometime, these tests will test those codec



-- 
This is an automated message from the 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-4992) Spark Row-writing Bulk Insert produces incorrect Bloom Filter metadata

2022-10-06 Thread Alexey Kudinkin (Jira)


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

Alexey Kudinkin updated HUDI-4992:
--
Status: Patch Available  (was: In Progress)

> Spark Row-writing Bulk Insert produces incorrect Bloom Filter metadata
> --
>
> Key: HUDI-4992
> URL: https://issues.apache.org/jira/browse/HUDI-4992
> Project: Apache Hudi
>  Issue Type: Bug
>Affects Versions: 0.12.0
>Reporter: Alexey Kudinkin
>Assignee: Alexey Kudinkin
>Priority: Blocker
>  Labels: pull-request-available
> Fix For: 0.12.1
>
>
> Troubleshooting duplicates issue w/ Abhishek Modi from Notion, we've found 
> that the min/max record key stats are being currently persisted incorrectly 
> into Parquet metadata, leading to duplicate records being produced in their 
> pipeline after initial bulk-insert.



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


[jira] [Updated] (HUDI-4992) Spark Row-writing Bulk Insert produces incorrect Bloom Filter metadata

2022-10-06 Thread Alexey Kudinkin (Jira)


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

Alexey Kudinkin updated HUDI-4992:
--
Sprint: 2022/10/04

> Spark Row-writing Bulk Insert produces incorrect Bloom Filter metadata
> --
>
> Key: HUDI-4992
> URL: https://issues.apache.org/jira/browse/HUDI-4992
> Project: Apache Hudi
>  Issue Type: Bug
>Affects Versions: 0.12.0
>Reporter: Alexey Kudinkin
>Assignee: Alexey Kudinkin
>Priority: Blocker
>  Labels: pull-request-available
> Fix For: 0.12.1
>
>
> Troubleshooting duplicates issue w/ Abhishek Modi from Notion, we've found 
> that the min/max record key stats are being currently persisted incorrectly 
> into Parquet metadata, leading to duplicate records being produced in their 
> pipeline after initial bulk-insert.



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


[GitHub] [hudi] hudi-bot commented on pull request #6883: [HUDI-4992] Fixing invalid min/max record key stats in Parquet metadata

2022-10-06 Thread GitBox


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

   
   ## CI report:
   
   * 024bcf68a222e98835e0b61fa784bc1040f65f62 Azure: 
[CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12037)
 
   * c528116cecfd35edc4b6829cb091c4924dc1758f Azure: 
[PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12038)
 
   
   
   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 #6883: [HUDI-4992] Fixing invalid min/max record key stats in Parquet metadata

2022-10-06 Thread GitBox


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

   
   ## CI report:
   
   * 024bcf68a222e98835e0b61fa784bc1040f65f62 Azure: 
[PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12037)
 
   * c528116cecfd35edc4b6829cb091c4924dc1758f 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 #6806: [HUDI-4905] Improve type handling in proto schema conversion

2022-10-06 Thread GitBox


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

   
   ## CI report:
   
   * 57f8b811946c7c013cbc31e9bd8db469f70cda2a Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12036)
 
   
   
   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-06 Thread GitBox


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

   
   ## CI report:
   
   * 00eefd74074b2e0e04dc308ab9b775e09ed7803b Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12035)
 
   
   
   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] alexeykudinkin commented on pull request #6883: [HUDI-4992] Fixing invalid min/max record key stats in Parquet metadata

2022-10-06 Thread GitBox


alexeykudinkin commented on PR #6883:
URL: https://github.com/apache/hudi/pull/6883#issuecomment-1270940646

   @nsivabalan yes, i'm adding comprehensive tests for Parquet writers to make 
sure we properly validate both records and metadata written to Parquet


-- 
This is an automated message from the 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] alexeykudinkin commented on a diff in pull request #6883: [HUDI-4992] Fixing invalid min/max record key stats in Parquet metadata

2022-10-06 Thread GitBox


alexeykudinkin commented on code in PR #6883:
URL: https://github.com/apache/hudi/pull/6883#discussion_r989585629


##
hudi-common/src/main/java/org/apache/hudi/avro/HoodieBloomFilterWriteSupport.java:
##
@@ -0,0 +1,78 @@
+package org.apache.hudi.avro;
+
+import org.apache.hudi.common.bloom.BloomFilter;
+import org.apache.hudi.common.bloom.HoodieDynamicBoundedBloomFilter;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import static 
org.apache.hudi.avro.HoodieAvroWriteSupport.HOODIE_AVRO_BLOOM_FILTER_METADATA_KEY;
+
+/**
+ * This is write-support utility base-class taking up handling of
+ *
+ * 
+ *   Adding record keys to the Bloom Filter
+ *   Keeping track of min/max record key (w/in single file)
+ * 
+ *
+ * @param  record-key type being ingested by this clas
+ */
+public abstract class HoodieBloomFilterWriteSupport> {
+
+  public static final String HOODIE_MIN_RECORD_KEY_FOOTER = 
"hoodie_min_record_key";
+  public static final String HOODIE_MAX_RECORD_KEY_FOOTER = 
"hoodie_max_record_key";
+  public static final String HOODIE_BLOOM_FILTER_TYPE_CODE = 
"hoodie_bloom_filter_type_code";
+
+  private final BloomFilter bloomFilter;
+
+  private T minRecordKey;
+  private T maxRecordKey;
+
+  public HoodieBloomFilterWriteSupport(BloomFilter bloomFilter) {
+this.bloomFilter = bloomFilter;
+  }
+
+  public void addKey(T recordKey) {
+bloomFilter.add(getUTF8Bytes(recordKey));

Review Comment:
   Bloom Filter always adds keys as UTF8, though previously we're just passing 
Strings into it (except in Row-writer)



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

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 pull request #6883: [HUDI-4992] Fixing invalid min/max record key stats in Parquet metadata

2022-10-06 Thread GitBox


nsivabalan commented on PR #6883:
URL: https://github.com/apache/hudi/pull/6883#issuecomment-1270864726

   can we please add tests as well


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

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 #6883: [HUDI-4992] Fixing invalid min/max record key stats in Parquet metadata

2022-10-06 Thread GitBox


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


##
hudi-common/src/main/java/org/apache/hudi/avro/HoodieBloomFilterWriteSupport.java:
##
@@ -0,0 +1,78 @@
+package org.apache.hudi.avro;
+
+import org.apache.hudi.common.bloom.BloomFilter;
+import org.apache.hudi.common.bloom.HoodieDynamicBoundedBloomFilter;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import static 
org.apache.hudi.avro.HoodieAvroWriteSupport.HOODIE_AVRO_BLOOM_FILTER_METADATA_KEY;
+
+/**
+ * This is write-support utility base-class taking up handling of
+ *
+ * 
+ *   Adding record keys to the Bloom Filter
+ *   Keeping track of min/max record key (w/in single file)
+ * 
+ *
+ * @param  record-key type being ingested by this clas
+ */
+public abstract class HoodieBloomFilterWriteSupport> {
+
+  public static final String HOODIE_MIN_RECORD_KEY_FOOTER = 
"hoodie_min_record_key";
+  public static final String HOODIE_MAX_RECORD_KEY_FOOTER = 
"hoodie_max_record_key";
+  public static final String HOODIE_BLOOM_FILTER_TYPE_CODE = 
"hoodie_bloom_filter_type_code";
+
+  private final BloomFilter bloomFilter;
+
+  private T minRecordKey;
+  private T maxRecordKey;
+
+  public HoodieBloomFilterWriteSupport(BloomFilter bloomFilter) {
+this.bloomFilter = bloomFilter;
+  }
+
+  public void addKey(T recordKey) {
+bloomFilter.add(getUTF8Bytes(recordKey));

Review Comment:
   previously, we were adding strings as is into bloom filter 
(HoodieAvroWriteSupport), but now w/ this patch, we are switching to adding 
UTF8Bytes? 



-- 
This is an automated message from the 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 #6883: [HUDI-4992] Fixing invalid min/max record key stats in Parquet metadata

2022-10-06 Thread GitBox


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

   
   ## CI report:
   
   * 024bcf68a222e98835e0b61fa784bc1040f65f62 Azure: 
[PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12037)
 
   
   
   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 #6883: [HUDI-4992] Fixing invalid min/max record key stats in Parquet metadata

2022-10-06 Thread GitBox


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

   
   ## CI report:
   
   * 024bcf68a222e98835e0b61fa784bc1040f65f62 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 #6575: [HUDI-4754] Add compliance check in github actions

2022-10-06 Thread GitBox


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

   
   ## CI report:
   
   * 1600e31836157c8d05e3bc8b9e08e1717471f1a6 UNKNOWN
   * 4d02f2c64a5fc4b89889677ee639a20b53cec26a UNKNOWN
   * 48147d19c835e7868102fd2d083659e6ee2ac343 UNKNOWN
   * b524fcc1dc3a5ce4d32a1238e09b9cc58b3e26b6 UNKNOWN
   * 3f2440a00e10b2c2daa4d930fd2933d48f5be1a2 UNKNOWN
   * 5dfc76a457a1ef80cc87d35a2bd24bab01edfd5b UNKNOWN
   * 580379dc4efdfdb6e0425738d54c91f3e20eedee Azure: 
[SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12034)
 
   
   
   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] CTTY commented on a diff in pull request #6882: [WIP][HUDI-3625] Object Store Storage Strategy

2022-10-06 Thread GitBox


CTTY commented on code in PR #6882:
URL: https://github.com/apache/hudi/pull/6882#discussion_r989554325


##
hudi-common/src/main/java/org/apache/hudi/common/table/view/HoodieTableFileSystemView.java:
##
@@ -167,6 +167,12 @@ public HoodieTableFileSystemView(HoodieTableMetaClient 
metaClient, HoodieTimelin
 addFilesToView(fileStatuses);
   }
 
+  public HoodieTableFileSystemView(HoodieTableMetaClient metaClient, 
HoodieTimeline visibleActiveTimeline,
+  Map partitionFiles) {
+this(metaClient, visibleActiveTimeline);
+addFilesToView(partitionFiles);
+  }
+

Review Comment:
   On the reading side, FSView has to be aware of physical path. Statuses got 
passed in should be referring to the actual file. But this breaks the logic of 
extracting partition name from file path (See L213 of 
[AbstractTableFileSystemView](https://github.com/apache/hudi/pull/6882/files#diff-201f1e7b1c7bce22fc142968e77cf67eea5c9d2946c8f8244d854fb6754b32a8))
 
   
   Thus I added this new constructor to pass the partition info down.  And this 
should replace most of usages of another constructor at L164



-- 
This is an automated message from the 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] CTTY commented on a diff in pull request #6882: [WIP][HUDI-3625] Object Store Storage Strategy

2022-10-06 Thread GitBox


CTTY commented on code in PR #6882:
URL: https://github.com/apache/hudi/pull/6882#discussion_r989554325


##
hudi-common/src/main/java/org/apache/hudi/common/table/view/HoodieTableFileSystemView.java:
##
@@ -167,6 +167,12 @@ public HoodieTableFileSystemView(HoodieTableMetaClient 
metaClient, HoodieTimelin
 addFilesToView(fileStatuses);
   }
 
+  public HoodieTableFileSystemView(HoodieTableMetaClient metaClient, 
HoodieTimeline visibleActiveTimeline,
+  Map partitionFiles) {
+this(metaClient, visibleActiveTimeline);
+addFilesToView(partitionFiles);
+  }
+

Review Comment:
   On the reading side, FSView has to be aware of physical path. Statuses got 
passed in should be referring to the actual file. But this breaks the logic of 
extracting partition name from file path (See L212 of 
[this](https://github.com/apache/hudi/pull/6882/files#diff-201f1e7b1c7bce22fc142968e77cf67eea5c9d2946c8f8244d854fb6754b32a8))
 
   
   Thus I added this new constructor to pass the partition info down.  And this 
should replace most of usages of another constructor at L164



-- 
This is an automated message from the 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] CTTY commented on a diff in pull request #6882: [HUDI-3625] Object Store Storage Strategy

2022-10-06 Thread GitBox


CTTY commented on code in PR #6882:
URL: https://github.com/apache/hudi/pull/6882#discussion_r989554325


##
hudi-common/src/main/java/org/apache/hudi/common/table/view/HoodieTableFileSystemView.java:
##
@@ -167,6 +167,12 @@ public HoodieTableFileSystemView(HoodieTableMetaClient 
metaClient, HoodieTimelin
 addFilesToView(fileStatuses);
   }
 
+  public HoodieTableFileSystemView(HoodieTableMetaClient metaClient, 
HoodieTimeline visibleActiveTimeline,
+  Map partitionFiles) {
+this(metaClient, visibleActiveTimeline);
+addFilesToView(partitionFiles);
+  }
+

Review Comment:
   On the reading side, FSView has to be aware of physical path. Statuses got 
passed in should be referring to the actual file. But this breaks the logic of 
extracting partition name from file path (See 
[this](https://github.com/apache/hudi/pull/6882/files#diff-201f1e7b1c7bce22fc142968e77cf67eea5c9d2946c8f8244d854fb6754b32a8R212))
 
   
   Thus I added this new constructor to pass the partition info down.  And this 
should replace most of usages of another constructor at L164



-- 
This is an automated message from the 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] CTTY commented on a diff in pull request #6882: [HUDI-3625] Object Store Storage Strategy

2022-10-06 Thread GitBox


CTTY commented on code in PR #6882:
URL: https://github.com/apache/hudi/pull/6882#discussion_r989554325


##
hudi-common/src/main/java/org/apache/hudi/common/table/view/HoodieTableFileSystemView.java:
##
@@ -167,6 +167,12 @@ public HoodieTableFileSystemView(HoodieTableMetaClient 
metaClient, HoodieTimelin
 addFilesToView(fileStatuses);
   }
 
+  public HoodieTableFileSystemView(HoodieTableMetaClient metaClient, 
HoodieTimeline visibleActiveTimeline,
+  Map partitionFiles) {
+this(metaClient, visibleActiveTimeline);
+addFilesToView(partitionFiles);
+  }
+

Review Comment:
   On the reading side, FSView has to be aware of physical path. Statuses got 
passed in should be referring to the actual file. But this is breaking the 
logic of getting partition name from file path (See L



-- 
This is an automated message from the 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] CTTY commented on a diff in pull request #6882: [HUDI-3625] Object Store Storage Strategy

2022-10-06 Thread GitBox


CTTY commented on code in PR #6882:
URL: https://github.com/apache/hudi/pull/6882#discussion_r989552176


##
hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSource.scala:
##
@@ -98,6 +98,67 @@ class TestCOWDataSource extends HoodieClientTestBase {
 System.gc()
   }
 
+  @Test def testFileLayout() {

Review Comment:
   Please ignore this for now



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

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

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



[GitHub] [hudi] CTTY commented on a diff in pull request #6882: [HUDI-3625] Object Store Storage Strategy

2022-10-06 Thread GitBox


CTTY commented on code in PR #6882:
URL: https://github.com/apache/hudi/pull/6882#discussion_r989551863


##
hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataPayload.java:
##
@@ -473,21 +479,26 @@ public Option 
getColumnStatMetadata() {
   /**
* Returns the files added as part of this record.
*/
-  public FileStatus[] getFileStatuses(Configuration hadoopConf, Path 
partitionPath) throws IOException {
-FileSystem fs = partitionPath.getFileSystem(hadoopConf);
-return getFileStatuses(fs, partitionPath);
-  }
+  public FileStatus[] getFileStatuses(FileSystem fs, String partitionPath,
+  HoodieStorageStrategy storageStrategy) {
+URI uri = fs.getUri();
+String scheme = uri.getScheme();
+String authority = uri.getAuthority();
 
-  /**
-   * Returns the files added as part of this record.
-   */
-  public FileStatus[] getFileStatuses(FileSystem fs, Path partitionPath) {
-long blockSize = fs.getDefaultBlockSize(partitionPath);
 return filterFileInfoEntries(false)
 .map(e -> {
+  String fileName = e.getKey();
+  String fileId = FSUtils.getFileId(fileName);
+
+  // Convert logical path to physical path
+  Path physicalPartitionPath = new Path(scheme, authority,
+  storageStrategy.storageLocation(partitionPath, fileId));
+

Review Comment:
   reading side path conversion. Only code that goes through metadata payload 
would be able to reach physical path with storage strategy



-- 
This is an automated message from the 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-4992) Spark Row-writing Bulk Insert produces incorrect Bloom Filter metadata

2022-10-06 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot updated HUDI-4992:
-
Labels: pull-request-available  (was: )

> Spark Row-writing Bulk Insert produces incorrect Bloom Filter metadata
> --
>
> Key: HUDI-4992
> URL: https://issues.apache.org/jira/browse/HUDI-4992
> Project: Apache Hudi
>  Issue Type: Bug
>Affects Versions: 0.12.0
>Reporter: Alexey Kudinkin
>Assignee: Alexey Kudinkin
>Priority: Blocker
>  Labels: pull-request-available
> Fix For: 0.12.1
>
>
> Troubleshooting duplicates issue w/ Abhishek Modi from Notion, we've found 
> that the min/max record key stats are being currently persisted incorrectly 
> into Parquet metadata, leading to duplicate records being produced in their 
> pipeline after initial bulk-insert.



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


[GitHub] [hudi] alexeykudinkin opened a new pull request, #6883: [HUDI-4992] Fixing invalid min/max record key stats in Parquet metadata

2022-10-06 Thread GitBox


alexeykudinkin opened a new pull request, #6883:
URL: https://github.com/apache/hudi/pull/6883

   ### Change Logs
   
   Troubleshooting duplicates issue, we've found that the min/max record key 
stats are being currently persisted incorrectly into Parquet metadata by 
Row-writing Bulk Insert, leading to duplicate records upon using the Bloom 
Filter subsequently.
   
   ### Impact
   
   **Risk level: Low**
   
   ### Documentation Update
   
   No docs update
   
   ### Contributor's checklist
   
   - [ ] Read through [contributor's 
guide](https://hudi.apache.org/contribute/how-to-contribute)
   - [ ] Change Logs and Impact were stated clearly
   - [ ] Adequate tests were added if applicable
   - [ ] CI passed
   


-- 
This is an automated message from the 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] CTTY commented on a diff in pull request #6882: [HUDI-3625] Object Store Storage Strategy

2022-10-06 Thread GitBox


CTTY commented on code in PR #6882:
URL: https://github.com/apache/hudi/pull/6882#discussion_r989549950


##
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java:
##
@@ -158,27 +161,29 @@ private String makeWriteToken() {
 return FSUtils.makeWriteToken(getPartitionId(), getStageId(), 
getAttemptId());
   }
 
-  public Path makeNewPath(String partitionPath) {
-Path path = FSUtils.getPartitionPath(config.getBasePath(), partitionPath);
+  public Path makeNewPhysicalPath(String partitionPath) {
+String storageLocation = storageStrategy.storageLocation(partitionPath, 
fileId);
+Path path = new Path(storageLocation);
 try {
   if (!fs.exists(path)) {
 fs.mkdirs(path); // create a new partition as needed.
   }
 } catch (IOException e) {
   throw new HoodieIOException("Failed to make dir " + path, e);
 }
+LOG.info("write handle, physical partition path: " + path);
 
-return new Path(path.toString(), FSUtils.makeBaseFileName(instantTime, 
writeToken, fileId,
+return new Path(storageLocation, FSUtils.makeBaseFileName(instantTime, 
writeToken, fileId,
 
hoodieTable.getMetaClient().getTableConfig().getBaseFileFormat().getFileExtension()));
   }
 
   /**
* Make new file path with given file name.
*/
-  protected Path makeNewFilePath(String partitionPath, String fileName) {
-String relativePath = new Path((partitionPath.isEmpty() ? "" : 
partitionPath + "/")
-+ fileName).toString();
-return new Path(config.getBasePath(), relativePath);
+  protected Path makeNewFilePhysicalPath(String partitionPath, String 
fileName) {
+String storageLocation = storageStrategy.storageLocation(partitionPath, 
fileId);
+
+return new Path(storageLocation, fileName);

Review Comment:
   Obtaining physical path on the writing side



-- 
This is an automated message from the 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-4992) Spark Row-writing Bulk Insert produces incorrect Bloom Filter metadata

2022-10-06 Thread Alexey Kudinkin (Jira)
Alexey Kudinkin created HUDI-4992:
-

 Summary: Spark Row-writing Bulk Insert produces incorrect Bloom 
Filter metadata
 Key: HUDI-4992
 URL: https://issues.apache.org/jira/browse/HUDI-4992
 Project: Apache Hudi
  Issue Type: Bug
Affects Versions: 0.12.0
Reporter: Alexey Kudinkin
Assignee: Alexey Kudinkin
 Fix For: 0.12.1


Troubleshooting duplicates issue w/ Abhishek Modi from Notion, we've found that 
the min/max record key stats are being currently persisted incorrectly into 
Parquet metadata, leading to duplicate records being produced in their pipeline 
after initial bulk-insert.



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


[jira] [Updated] (HUDI-4992) Spark Row-writing Bulk Insert produces incorrect Bloom Filter metadata

2022-10-06 Thread Alexey Kudinkin (Jira)


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

Alexey Kudinkin updated HUDI-4992:
--
Status: In Progress  (was: Open)

> Spark Row-writing Bulk Insert produces incorrect Bloom Filter metadata
> --
>
> Key: HUDI-4992
> URL: https://issues.apache.org/jira/browse/HUDI-4992
> Project: Apache Hudi
>  Issue Type: Bug
>Affects Versions: 0.12.0
>Reporter: Alexey Kudinkin
>Assignee: Alexey Kudinkin
>Priority: Blocker
> Fix For: 0.12.1
>
>
> Troubleshooting duplicates issue w/ Abhishek Modi from Notion, we've found 
> that the min/max record key stats are being currently persisted incorrectly 
> into Parquet metadata, leading to duplicate records being produced in their 
> pipeline after initial bulk-insert.



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


[GitHub] [hudi] CTTY commented on a diff in pull request #6882: [HUDI-3625] Object Store Storage Strategy

2022-10-06 Thread GitBox


CTTY commented on code in PR #6882:
URL: https://github.com/apache/hudi/pull/6882#discussion_r989549646


##
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieCreateHandle.java:
##
@@ -235,10 +234,10 @@ protected void setupWriteStatus() throws IOException {
 stat.setNumInserts(insertRecordsWritten);
 stat.setPrevCommit(HoodieWriteStat.NULL_COMMIT);
 stat.setFileId(writeStatus.getFileId());
-stat.setPath(new Path(config.getBasePath()), path);
+stat.setPath(FSUtils.getLogicalRelativeFilePathStr(partitionPath, 
physicalPath.getName()));

Review Comment:
   In `HoodieCommitMetadata` / `HoodieWriteStat`, Hudi would still store 
logical path there



-- 
This is an automated message from the 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] CTTY commented on a diff in pull request #6882: [HUDI-3625] Object Store Storage Strategy

2022-10-06 Thread GitBox


CTTY commented on code in PR #6882:
URL: https://github.com/apache/hudi/pull/6882#discussion_r989548780


##
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieCreateHandle.java:
##
@@ -92,19 +92,18 @@ public HoodieCreateHandle(HoodieWriteConfig config, String 
instantTime, HoodieTa
 writeStatus.setFileId(fileId);
 writeStatus.setPartitionPath(partitionPath);
 writeStatus.setStat(new HoodieWriteStat());
-
-this.path = makeNewPath(partitionPath);
+this.physicalPath = makeNewPhysicalPath(partitionPath);
 
 try {
   HoodiePartitionMetadata partitionMetadata = new 
HoodiePartitionMetadata(fs, instantTime,
   new Path(config.getBasePath()), 
FSUtils.getPartitionPath(config.getBasePath(), partitionPath),
   hoodieTable.getPartitionMetafileFormat());
   partitionMetadata.trySave(getPartitionId());
   createMarkerFile(partitionPath, 
FSUtils.makeBaseFileName(this.instantTime, this.writeToken, this.fileId, 
hoodieTable.getBaseFileExtension()));
-  this.fileWriter = HoodieFileWriterFactory.getFileWriter(instantTime, 
path, hoodieTable, config,
+  this.fileWriter = HoodieFileWriterFactory.getFileWriter(instantTime, 
physicalPath, hoodieTable, config,

Review Comment:
   Physical path would be used to initialize file writer



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

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 #6806: [HUDI-4905] Improve type handling in proto schema conversion

2022-10-06 Thread GitBox


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

   
   ## CI report:
   
   * f03f9610cf4e2c490d33ca734ca9b3241b2be778 Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12032)
 Azure: 
[CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12012)
 
   * 57f8b811946c7c013cbc31e9bd8db469f70cda2a Azure: 
[PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12036)
 
   
   
   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-06 Thread GitBox


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

   
   ## CI report:
   
   * bfb3668bfbeb8c461e7d5293b67aac32be59d8ab Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11994)
 
   * 00eefd74074b2e0e04dc308ab9b775e09ed7803b Azure: 
[PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12035)
 
   
   
   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 #6806: [HUDI-4905] Improve type handling in proto schema conversion

2022-10-06 Thread GitBox


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

   
   ## CI report:
   
   * f03f9610cf4e2c490d33ca734ca9b3241b2be778 Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12032)
 Azure: 
[CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12012)
 
   * 57f8b811946c7c013cbc31e9bd8db469f70cda2a 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-06 Thread GitBox


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

   
   ## CI report:
   
   * bfb3668bfbeb8c461e7d5293b67aac32be59d8ab Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11994)
 
   * 00eefd74074b2e0e04dc308ab9b775e09ed7803b 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 #6882: [HUDI-3625] Object Store Storage Strategy

2022-10-06 Thread GitBox


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

   
   ## CI report:
   
   * 1b456c26360a564f7695b2f683c929add7b55daf Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12033)
 
   
   
   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 #6806: [HUDI-4905] Improve type handling in proto schema conversion

2022-10-06 Thread GitBox


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

   
   ## CI report:
   
   * f03f9610cf4e2c490d33ca734ca9b3241b2be778 Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12032)
 Azure: 
[CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12012)
 
   
   
   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-4937) Fix HoodieTable injecting HoodieBackedTableMetadat not reusing underlying MT readers

2022-10-06 Thread Alexey Kudinkin (Jira)


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

Alexey Kudinkin updated HUDI-4937:
--
Reviewers: sivabalan narayanan

> Fix HoodieTable injecting HoodieBackedTableMetadat not reusing underlying MT 
> readers
> 
>
> Key: HUDI-4937
> URL: https://issues.apache.org/jira/browse/HUDI-4937
> Project: Apache Hudi
>  Issue Type: Bug
>  Components: reader-core, writer-core
>Affects Versions: 0.12.0
>Reporter: Alexey Kudinkin
>Assignee: Alexey Kudinkin
>Priority: Blocker
>  Labels: pull-request-available
> Fix For: 0.13.0
>
>
> Currently, `HoodieTable` is holding `HoodieBackedTableMetadata` that is setup 
> not to reuse actual LogScanner and HFileReader used to read MT itself.
> This is proving to be wasteful on a number of occasions already, including 
> (not an exhaustive list):
> https://github.com/apache/hudi/issues/6373



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


[GitHub] [hudi] hudi-bot commented on pull request #6575: [HUDI-4754] Add compliance check in github actions

2022-10-06 Thread GitBox


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

   
   ## CI report:
   
   * 1600e31836157c8d05e3bc8b9e08e1717471f1a6 UNKNOWN
   * 4d02f2c64a5fc4b89889677ee639a20b53cec26a UNKNOWN
   * 48147d19c835e7868102fd2d083659e6ee2ac343 UNKNOWN
   * b524fcc1dc3a5ce4d32a1238e09b9cc58b3e26b6 UNKNOWN
   * 3f2440a00e10b2c2daa4d930fd2933d48f5be1a2 UNKNOWN
   * 5dfc76a457a1ef80cc87d35a2bd24bab01edfd5b UNKNOWN
   * 7e3c8e9cd3f9b8c37995c9819676a64bd8c89e77 Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11976)
 
   * 580379dc4efdfdb6e0425738d54c91f3e20eedee Azure: 
[PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12034)
 
   
   
   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-4937) Fix HoodieTable injecting HoodieBackedTableMetadat not reusing underlying MT readers

2022-10-06 Thread Alexey Kudinkin (Jira)


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

Alexey Kudinkin updated HUDI-4937:
--
Priority: Blocker  (was: Critical)

> Fix HoodieTable injecting HoodieBackedTableMetadat not reusing underlying MT 
> readers
> 
>
> Key: HUDI-4937
> URL: https://issues.apache.org/jira/browse/HUDI-4937
> Project: Apache Hudi
>  Issue Type: Bug
>  Components: reader-core, writer-core
>Affects Versions: 0.12.0
>Reporter: Alexey Kudinkin
>Assignee: Alexey Kudinkin
>Priority: Blocker
>  Labels: pull-request-available
> Fix For: 0.13.0
>
>
> Currently, `HoodieTable` is holding `HoodieBackedTableMetadata` that is setup 
> not to reuse actual LogScanner and HFileReader used to read MT itself.
> This is proving to be wasteful on a number of occasions already, including 
> (not an exhaustive list):
> https://github.com/apache/hudi/issues/6373



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


[jira] [Updated] (HUDI-4937) Fix HoodieTable injecting HoodieBackedTableMetadat not reusing underlying MT readers

2022-10-06 Thread Alexey Kudinkin (Jira)


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

Alexey Kudinkin updated HUDI-4937:
--
Status: Patch Available  (was: In Progress)

> Fix HoodieTable injecting HoodieBackedTableMetadat not reusing underlying MT 
> readers
> 
>
> Key: HUDI-4937
> URL: https://issues.apache.org/jira/browse/HUDI-4937
> Project: Apache Hudi
>  Issue Type: Bug
>  Components: reader-core, writer-core
>Affects Versions: 0.12.0
>Reporter: Alexey Kudinkin
>Assignee: Alexey Kudinkin
>Priority: Critical
>  Labels: pull-request-available
> Fix For: 0.13.0
>
>
> Currently, `HoodieTable` is holding `HoodieBackedTableMetadata` that is setup 
> not to reuse actual LogScanner and HFileReader used to read MT itself.
> This is proving to be wasteful on a number of occasions already, including 
> (not an exhaustive list):
> https://github.com/apache/hudi/issues/6373



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


[jira] [Updated] (HUDI-4937) Fix HoodieTable injecting HoodieBackedTableMetadat not reusing underlying MT readers

2022-10-06 Thread Alexey Kudinkin (Jira)


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

Alexey Kudinkin updated HUDI-4937:
--
Status: In Progress  (was: Open)

> Fix HoodieTable injecting HoodieBackedTableMetadat not reusing underlying MT 
> readers
> 
>
> Key: HUDI-4937
> URL: https://issues.apache.org/jira/browse/HUDI-4937
> Project: Apache Hudi
>  Issue Type: Bug
>  Components: reader-core, writer-core
>Affects Versions: 0.12.0
>Reporter: Alexey Kudinkin
>Assignee: Alexey Kudinkin
>Priority: Critical
>  Labels: pull-request-available
> Fix For: 0.13.0
>
>
> Currently, `HoodieTable` is holding `HoodieBackedTableMetadata` that is setup 
> not to reuse actual LogScanner and HFileReader used to read MT itself.
> This is proving to be wasteful on a number of occasions already, including 
> (not an exhaustive list):
> https://github.com/apache/hudi/issues/6373



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


[jira] [Updated] (HUDI-4812) Lazy partition listing and file groups fetching in Spark Query

2022-10-06 Thread Alexey Kudinkin (Jira)


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

Alexey Kudinkin updated HUDI-4812:
--
Status: In Progress  (was: Open)

> Lazy partition listing and file groups fetching in Spark Query
> --
>
> Key: HUDI-4812
> URL: https://issues.apache.org/jira/browse/HUDI-4812
> Project: Apache Hudi
>  Issue Type: Improvement
>  Components: spark
>Reporter: Yuwei Xiao
>Assignee: Yuwei Xiao
>Priority: Blocker
>  Labels: pull-request-available
> Fix For: 0.13.0
>
>
> In current spark query implementation, the FileIndex will refresh and load 
> all file groups in cached in order to serve subsequent queries.
>  
> For large table with many partitions, this may introduce much overhead in 
> initialization. Meanwhile, the query itself may come with partition filter. 
> So the loading of file groups will be unnecessary.
>  
> So to optimize, the whole refresh logic will become lazy, where actual work 
> will be carried out only after the partition filter.



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


[jira] [Updated] (HUDI-3963) [RFC-53] Use Lock-Free Message Queue Improving Hoodie Writing Efficiency

2022-10-06 Thread Alexey Kudinkin (Jira)


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

Alexey Kudinkin updated HUDI-3963:
--
Status: In Progress  (was: Open)

> [RFC-53] Use Lock-Free Message Queue Improving Hoodie Writing Efficiency
> 
>
> Key: HUDI-3963
> URL: https://issues.apache.org/jira/browse/HUDI-3963
> Project: Apache Hudi
>  Issue Type: New Feature
>Reporter: Yue Zhang
>Assignee: Yue Zhang
>Priority: Blocker
>  Labels: pull-request-available
> Fix For: 0.13.0
>
>




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


[jira] [Updated] (HUDI-4489) Break down HoodieAnalysis rules into Spark-specific components

2022-10-06 Thread Alexey Kudinkin (Jira)


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

Alexey Kudinkin updated HUDI-4489:
--
Status: In Progress  (was: Open)

> Break down HoodieAnalysis rules into Spark-specific components
> --
>
> Key: HUDI-4489
> URL: https://issues.apache.org/jira/browse/HUDI-4489
> Project: Apache Hudi
>  Issue Type: Bug
>Reporter: Alexey Kudinkin
>Assignee: Alexey Kudinkin
>Priority: Blocker
> Fix For: 0.13.0
>
> Attachments: image-2022-07-27-14-24-03-190.png
>
>
> Currently, `HoodieAnalysis` hosts Rules that are going to be universally 
> applied to all Spark versions even though they don't have to:
> For ex, Hudi back-ports `DeleteFromTable`, `UpdateTable` commands to Spark 2, 
> requiring corresponding resolution to occur:
> !image-2022-07-27-14-24-03-190.png|width=512,height=144!
>  
> However, this rule should not be applied for Spark 3, which has its own rules 
> applied to DeleteFromTable alas other commands



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


[jira] [Updated] (HUDI-3963) [RFC-53] Use Lock-Free Message Queue Improving Hoodie Writing Efficiency

2022-10-06 Thread Alexey Kudinkin (Jira)


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

Alexey Kudinkin updated HUDI-3963:
--
Status: Patch Available  (was: In Progress)

> [RFC-53] Use Lock-Free Message Queue Improving Hoodie Writing Efficiency
> 
>
> Key: HUDI-3963
> URL: https://issues.apache.org/jira/browse/HUDI-3963
> Project: Apache Hudi
>  Issue Type: New Feature
>Reporter: Yue Zhang
>Assignee: Yue Zhang
>Priority: Blocker
>  Labels: pull-request-available
> Fix For: 0.13.0
>
>




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


[jira] [Updated] (HUDI-4812) Lazy partition listing and file groups fetching in Spark Query

2022-10-06 Thread Alexey Kudinkin (Jira)


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

Alexey Kudinkin updated HUDI-4812:
--
Status: Patch Available  (was: In Progress)

> Lazy partition listing and file groups fetching in Spark Query
> --
>
> Key: HUDI-4812
> URL: https://issues.apache.org/jira/browse/HUDI-4812
> Project: Apache Hudi
>  Issue Type: Improvement
>  Components: spark
>Reporter: Yuwei Xiao
>Assignee: Yuwei Xiao
>Priority: Blocker
>  Labels: pull-request-available
> Fix For: 0.13.0
>
>
> In current spark query implementation, the FileIndex will refresh and load 
> all file groups in cached in order to serve subsequent queries.
>  
> For large table with many partitions, this may introduce much overhead in 
> initialization. Meanwhile, the query itself may come with partition filter. 
> So the loading of file groups will be unnecessary.
>  
> So to optimize, the whole refresh logic will become lazy, where actual work 
> will be carried out only after the partition filter.



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


[jira] [Updated] (HUDI-4472) Revisit schema handling in HoodieSparkSqlWriter

2022-10-06 Thread Alexey Kudinkin (Jira)


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

Alexey Kudinkin updated HUDI-4472:
--
Status: Patch Available  (was: In Progress)

> Revisit schema handling in HoodieSparkSqlWriter
> ---
>
> Key: HUDI-4472
> URL: https://issues.apache.org/jira/browse/HUDI-4472
> Project: Apache Hudi
>  Issue Type: Bug
>  Components: writer-core
>Affects Versions: 0.12.0
>Reporter: Alexey Kudinkin
>Assignee: Alexey Kudinkin
>Priority: Blocker
> Fix For: 0.13.0
>
>
> After many features aimed to bring more and more sophisticated support of 
> schema evolution were layered in w/in HoodieSparkSqlWriter, it's currently 
> requiring careful attention to reconcile many flows and make sure that the 
> original invariants still hold.
>  
> One example of the issue was discovered while addressing HUDI-4081 (which was 
> duct-typed in [#6213|https://github.com/apache/hudi/pull/6213/files#] to 
> avoid substantial changes before the release)



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


[GitHub] [hudi] hudi-bot commented on pull request #6575: [HUDI-4754] Add compliance check in github actions

2022-10-06 Thread GitBox


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

   
   ## CI report:
   
   * 1600e31836157c8d05e3bc8b9e08e1717471f1a6 UNKNOWN
   * 4d02f2c64a5fc4b89889677ee639a20b53cec26a UNKNOWN
   * 48147d19c835e7868102fd2d083659e6ee2ac343 UNKNOWN
   * b524fcc1dc3a5ce4d32a1238e09b9cc58b3e26b6 UNKNOWN
   * 3f2440a00e10b2c2daa4d930fd2933d48f5be1a2 UNKNOWN
   * 5dfc76a457a1ef80cc87d35a2bd24bab01edfd5b UNKNOWN
   * 7e3c8e9cd3f9b8c37995c9819676a64bd8c89e77 Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11976)
 
   * 580379dc4efdfdb6e0425738d54c91f3e20eedee 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] alexeykudinkin commented on issue #6842: HoodieDeltaStreamer is not honoring schema registry SSL key/trust store properties when connecting schema.registry.url

2022-10-06 Thread GitBox


alexeykudinkin commented on issue #6842:
URL: https://github.com/apache/hudi/issues/6842#issuecomment-1270699265

   Created HUDI-4991 to track


-- 
This is an automated message from the 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-4991) Make sure DeltaStreamer passes SSL key/truststore configs connecting to Schema Registry

2022-10-06 Thread Alexey Kudinkin (Jira)


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

Alexey Kudinkin updated HUDI-4991:
--
Sprint: 2022/10/04

> Make sure DeltaStreamer passes SSL key/truststore configs connecting to 
> Schema Registry
> ---
>
> Key: HUDI-4991
> URL: https://issues.apache.org/jira/browse/HUDI-4991
> Project: Apache Hudi
>  Issue Type: Bug
>  Components: deltastreamer
>Reporter: Alexey Kudinkin
>Assignee: Alexey Kudinkin
>Priority: Critical
> Fix For: 0.13.0
>
>
> Originally reported at:
> [https://github.com/apache/hudi/issues/6842]
>  
> Whenever Schema Registry is used requiring passing keystore/truststore params 
> to access SSL certificates (like below) DeltaStreamer fails:
> {code:java}
> mode.hoodie.deltastreamer.schemaprovider.registry.url=https://schemaregistry.com
> schema.registry.ssl.keystore.location=/artifacts/topics/certs/keystore.jks
> schema.registry.ssl.keystore.password=
> schema.registry.ssl.truststore.location=/artifacts/topics/certs/truststore.jks
> schema.registry.ssl.truststore.password=
> schema.registry.ssl.key.password= {code}
> {code:java}
> at 
> org.apache.hudi.utilities.schema.SchemaRegistryProvider.getSourceSchema(SchemaRegistryProvider.java:109)
> at 
> org.apache.hudi.utilities.schema.SchemaProviderWithPostProcessor.lambda$getSourceSchema$0(SchemaProviderWithPostProcessor.java:41)
> at org.apache.hudi.common.util.Option.map(Option.java:108)
> at 
> org.apache.hudi.utilities.schema.SchemaProviderWithPostProcessor.getSourceSchema(SchemaProviderWithPostProcessor.java:41)
> at 
> org.apache.hudi.utilities.deltastreamer.DeltaSync.registerAvroSchemas(DeltaSync.java:839)
> at 
> org.apache.hudi.utilities.deltastreamer.DeltaSync.(DeltaSync.java:233)
> at 
> org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer$DeltaSyncService.(HoodieDeltaStreamer.java:646)
> at 
> org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer.(HoodieDeltaStreamer.java:142)
> at 
> org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer.(HoodieDeltaStreamer.java:115)
> at 
> org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer.main(HoodieDeltaStreamer.java:549)
> 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.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)
> at 
> org.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:1000)
> at org.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)
> at org.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)
> at org.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)
> at 
> org.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1089)
> at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1098)
> at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)
> Caused by: javax.net.ssl.SSLHandshakeException: PKIX path building failed: 
> sun.security.provider.certpath.SunCertPathBuilderException: unable to find 
> valid certification path to requested target
> at sun.security.ssl.Alert.createSSLException(Alert.java:131)
> at sun.security.ssl.TransportContext.fatal(TransportContext.java:324)
> at sun.security.ssl.TransportContext.fatal(TransportContext.java:267)
> at sun.security.ssl.TransportContext.fatal(TransportContext.java:262)
> at 
> sun.security.ssl.CertificateMessage$T12CertificateConsumer.checkServerCerts(CertificateMessage.java:654)
> at 
> sun.security.ssl.CertificateMessage$T12CertificateConsumer.onCertificate(CertificateMessage.java:473)
> at 
> sun.security.ssl.CertificateMessage$T12CertificateConsumer.consume(CertificateMessage.java:369)
> at sun.security.ssl.SSLHandshake.consume(SSLHandshake.java:377)
> at sun.security.ssl.HandshakeContext.dispatch(HandshakeContext.java:444)
> at sun.security.ssl.HandshakeContext.dispatch(HandshakeContext.java:422)
> at sun.security.ssl.TransportContext.dispatch(TransportContext.java:182)
> at sun.security.ssl.SSLTransport.decode(SSLTransport.java:152)
> at sun.security.ssl.SSLSocketImpl.decode(SSLSocketImpl.java:1397)
> at 
> sun.security.ssl.SSLSocketImpl.readHandshakeRecord(SSLSocketImpl.java:1305)
> at sun.security.ssl.SSLSocketImpl.startHandshake(SSLSocketImpl.java:440)
> at 
> sun.net.www.protocol.https.HttpsClient.afterConnect(HttpsClient.java:559)
> at 
> 

[jira] [Created] (HUDI-4991) Make sure DeltaStreamer passes SSL key/truststore configs connecting to Schema Registry

2022-10-06 Thread Alexey Kudinkin (Jira)
Alexey Kudinkin created HUDI-4991:
-

 Summary: Make sure DeltaStreamer passes SSL key/truststore configs 
connecting to Schema Registry
 Key: HUDI-4991
 URL: https://issues.apache.org/jira/browse/HUDI-4991
 Project: Apache Hudi
  Issue Type: Bug
  Components: deltastreamer
Reporter: Alexey Kudinkin
Assignee: Alexey Kudinkin
 Fix For: 0.13.0


Originally reported at:

[https://github.com/apache/hudi/issues/6842]

 

Whenever Schema Registry is used requiring passing keystore/truststore params 
to access SSL certificates (like below) DeltaStreamer fails:
{code:java}
mode.hoodie.deltastreamer.schemaprovider.registry.url=https://schemaregistry.com
schema.registry.ssl.keystore.location=/artifacts/topics/certs/keystore.jks
schema.registry.ssl.keystore.password=
schema.registry.ssl.truststore.location=/artifacts/topics/certs/truststore.jks
schema.registry.ssl.truststore.password=
schema.registry.ssl.key.password= {code}
{code:java}
at 
org.apache.hudi.utilities.schema.SchemaRegistryProvider.getSourceSchema(SchemaRegistryProvider.java:109)
at 
org.apache.hudi.utilities.schema.SchemaProviderWithPostProcessor.lambda$getSourceSchema$0(SchemaProviderWithPostProcessor.java:41)
at org.apache.hudi.common.util.Option.map(Option.java:108)
at 
org.apache.hudi.utilities.schema.SchemaProviderWithPostProcessor.getSourceSchema(SchemaProviderWithPostProcessor.java:41)
at 
org.apache.hudi.utilities.deltastreamer.DeltaSync.registerAvroSchemas(DeltaSync.java:839)
at 
org.apache.hudi.utilities.deltastreamer.DeltaSync.(DeltaSync.java:233)
at 
org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer$DeltaSyncService.(HoodieDeltaStreamer.java:646)
at 
org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer.(HoodieDeltaStreamer.java:142)
at 
org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer.(HoodieDeltaStreamer.java:115)
at 
org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer.main(HoodieDeltaStreamer.java:549)
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.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)
at 
org.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:1000)
at org.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)
at org.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)
at org.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)
at 
org.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1089)
at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1098)
at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)
Caused by: javax.net.ssl.SSLHandshakeException: PKIX path building failed: 
sun.security.provider.certpath.SunCertPathBuilderException: unable to find 
valid certification path to requested target
at sun.security.ssl.Alert.createSSLException(Alert.java:131)
at sun.security.ssl.TransportContext.fatal(TransportContext.java:324)
at sun.security.ssl.TransportContext.fatal(TransportContext.java:267)
at sun.security.ssl.TransportContext.fatal(TransportContext.java:262)
at 
sun.security.ssl.CertificateMessage$T12CertificateConsumer.checkServerCerts(CertificateMessage.java:654)
at 
sun.security.ssl.CertificateMessage$T12CertificateConsumer.onCertificate(CertificateMessage.java:473)
at 
sun.security.ssl.CertificateMessage$T12CertificateConsumer.consume(CertificateMessage.java:369)
at sun.security.ssl.SSLHandshake.consume(SSLHandshake.java:377)
at sun.security.ssl.HandshakeContext.dispatch(HandshakeContext.java:444)
at sun.security.ssl.HandshakeContext.dispatch(HandshakeContext.java:422)
at sun.security.ssl.TransportContext.dispatch(TransportContext.java:182)
at sun.security.ssl.SSLTransport.decode(SSLTransport.java:152)
at sun.security.ssl.SSLSocketImpl.decode(SSLSocketImpl.java:1397)
at 
sun.security.ssl.SSLSocketImpl.readHandshakeRecord(SSLSocketImpl.java:1305)
at sun.security.ssl.SSLSocketImpl.startHandshake(SSLSocketImpl.java:440)
at sun.net.www.protocol.https.HttpsClient.afterConnect(HttpsClient.java:559)
at 
sun.net.www.protocol.https.AbstractDelegateHttpsURLConnection.connect(AbstractDelegateHttpsURLConnection.java:197)
at 
sun.net.www.protocol.http.HttpURLConnection.getInputStream0(HttpURLConnection.java:1572)
at 
sun.net.www.protocol.http.HttpURLConnection.getInputStream(HttpURLConnection.java:1500)
at 
sun.net.www.protocol.https.HttpsURLConnectionImpl.getInputStream(HttpsURLConnectionImpl.java:268)

[GitHub] [hudi] hudi-bot commented on pull request #6882: [HUDI-3625] Object Store Storage Strategy

2022-10-06 Thread GitBox


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

   
   ## CI report:
   
   * 1b456c26360a564f7695b2f683c929add7b55daf Azure: 
[PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12033)
 
   
   
   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 #6806: [HUDI-4905] Improve type handling in proto schema conversion

2022-10-06 Thread GitBox


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

   
   ## CI report:
   
   * f03f9610cf4e2c490d33ca734ca9b3241b2be778 Azure: 
[PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12032)
 Azure: 
[CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12012)
 
   
   
   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 #6882: [HUDI-3625] Object Store Storage Strategy

2022-10-06 Thread GitBox


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

   
   ## CI report:
   
   * 1b456c26360a564f7695b2f683c929add7b55daf 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 #6806: [HUDI-4905] Improve type handling in proto schema conversion

2022-10-06 Thread GitBox


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

   
   ## CI report:
   
   * f03f9610cf4e2c490d33ca734ca9b3241b2be778 Azure: 
[PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12012)
 Azure: 
[PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12032)
 
   *  Unknown: [CANCELED](TBD) 
   
   
   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 #6806: [HUDI-4905] Improve type handling in proto schema conversion

2022-10-06 Thread GitBox


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

   
   ## CI report:
   
   * f03f9610cf4e2c490d33ca734ca9b3241b2be778 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] CTTY commented on pull request #6882: [HUDI-3625] Object Store Storage Strategy

2022-10-06 Thread GitBox


CTTY commented on PR #6882:
URL: https://github.com/apache/hudi/pull/6882#issuecomment-1270619909

   See [RFC-60](https://github.com/apache/hudi/pull/5113)


-- 
This is an automated message from the 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] CTTY opened a new pull request, #6882: [HUDI-3625] Object Store Storage Strategy

2022-10-06 Thread GitBox


CTTY opened a new pull request, #6882:
URL: https://github.com/apache/hudi/pull/6882

   ### Change Logs
   
   _Describe context and summary for this change. Highlight if any code was 
copied._
   
   ### Impact
   
   _Describe any public API or user-facing feature change or any performance 
impact._
   
   **Risk level: none | low | medium | high**
   
   _Choose one. If medium or high, explain what verification was done to 
mitigate the risks._
   
   ### Documentation Update
   
   _Describe any necessary documentation update if there is any new feature, 
config, or user-facing change_
   
   - _The config description must be updated if new configs are added or the 
default value of the configs are changed_
   - _Any new feature or user-facing change requires updating the Hudi website. 
Please create a Jira ticket, attach the
 ticket number here and follow the 
[instruction](https://hudi.apache.org/contribute/developer-setup#website) to 
make
 changes to the website._
   
   ### Contributor's checklist
   
   - [ ] Read through [contributor's 
guide](https://hudi.apache.org/contribute/how-to-contribute)
   - [ ] Change Logs and Impact were stated clearly
   - [ ] Adequate tests were added if applicable
   - [ ] CI passed
   


-- 
This is an automated message from the 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] kazdy commented on issue #6869: [SUPPORT] Incremental upsert or merge is not working

2022-10-06 Thread GitBox


kazdy commented on issue #6869:
URL: https://github.com/apache/hudi/issues/6869#issuecomment-1270599382

   Hi, 
   
   record key works like a PK on a table (unique, non-nullable field). 
   In your case, you end up with two records with different record key and 
that's expected.
   Precombine and upserts are supposed to maintain the uniqueness of recordKey.
   
   assume you use only delivery field as record key to make it easier 
   so if you have record with delivery:3000 hudi will do insert (if record with 
same record key does not exists in the table),  if record with delivery:2000 
and it already exists in the table then update
   
   precombine works before write, incoming batch of data is deduplicated based 
on record key and precombine field
   so if in incoming batch you have two records with the same record key, then 
one with greater precombine field value will be passed to write operation


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

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

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



[hudi] branch release-0.12.1 updated: [MINOR] Update release version to reflect published version 0.12.1

2022-10-06 Thread yuzhaojing
This is an automated email from the ASF dual-hosted git repository.

yuzhaojing pushed a commit to branch release-0.12.1
in repository https://gitbox.apache.org/repos/asf/hudi.git


The following commit(s) were added to refs/heads/release-0.12.1 by this push:
 new 28cb191df7 [MINOR] Update release version to reflect published version 
 0.12.1
28cb191df7 is described below

commit 28cb191df76ae2626b9d712a17af576212dc82f0
Author: 喻兆靖 
AuthorDate: Fri Oct 7 03:27:10 2022 +0800

[MINOR] Update release version to reflect published version  0.12.1
---
 docker/hoodie/hadoop/base/pom.xml  | 2 +-
 docker/hoodie/hadoop/base_java11/pom.xml   | 2 +-
 docker/hoodie/hadoop/datanode/pom.xml  | 2 +-
 docker/hoodie/hadoop/historyserver/pom.xml | 2 +-
 docker/hoodie/hadoop/hive_base/pom.xml | 2 +-
 docker/hoodie/hadoop/namenode/pom.xml  | 2 +-
 docker/hoodie/hadoop/pom.xml   | 2 +-
 docker/hoodie/hadoop/prestobase/pom.xml| 2 +-
 docker/hoodie/hadoop/spark_base/pom.xml| 2 +-
 docker/hoodie/hadoop/sparkadhoc/pom.xml| 2 +-
 docker/hoodie/hadoop/sparkmaster/pom.xml   | 2 +-
 docker/hoodie/hadoop/sparkworker/pom.xml   | 2 +-
 docker/hoodie/hadoop/trinobase/pom.xml | 2 +-
 docker/hoodie/hadoop/trinocoordinator/pom.xml  | 2 +-
 docker/hoodie/hadoop/trinoworker/pom.xml   | 2 +-
 hudi-aws/pom.xml   | 4 ++--
 hudi-cli/pom.xml   | 2 +-
 hudi-client/hudi-client-common/pom.xml | 4 ++--
 hudi-client/hudi-flink-client/pom.xml  | 4 ++--
 hudi-client/hudi-java-client/pom.xml   | 4 ++--
 hudi-client/hudi-spark-client/pom.xml  | 4 ++--
 hudi-client/pom.xml| 2 +-
 hudi-common/pom.xml| 2 +-
 hudi-examples/hudi-examples-common/pom.xml | 2 +-
 hudi-examples/hudi-examples-flink/pom.xml  | 2 +-
 hudi-examples/hudi-examples-java/pom.xml   | 2 +-
 hudi-examples/hudi-examples-spark/pom.xml  | 2 +-
 hudi-examples/pom.xml  | 2 +-
 hudi-flink-datasource/hudi-flink/pom.xml   | 4 ++--
 hudi-flink-datasource/hudi-flink1.13.x/pom.xml | 4 ++--
 hudi-flink-datasource/hudi-flink1.14.x/pom.xml | 4 ++--
 hudi-flink-datasource/hudi-flink1.15.x/pom.xml | 4 ++--
 hudi-flink-datasource/pom.xml  | 4 ++--
 hudi-gcp/pom.xml   | 2 +-
 hudi-hadoop-mr/pom.xml | 2 +-
 hudi-integ-test/pom.xml| 2 +-
 hudi-kafka-connect/pom.xml | 4 ++--
 hudi-spark-datasource/hudi-spark-common/pom.xml| 4 ++--
 hudi-spark-datasource/hudi-spark/pom.xml   | 4 ++--
 hudi-spark-datasource/hudi-spark2-common/pom.xml   | 2 +-
 hudi-spark-datasource/hudi-spark2/pom.xml  | 4 ++--
 hudi-spark-datasource/hudi-spark3-common/pom.xml   | 2 +-
 hudi-spark-datasource/hudi-spark3.1.x/pom.xml  | 4 ++--
 hudi-spark-datasource/hudi-spark3.2.x/pom.xml  | 4 ++--
 hudi-spark-datasource/hudi-spark3.2plus-common/pom.xml | 2 +-
 hudi-spark-datasource/hudi-spark3.3.x/pom.xml  | 4 ++--
 hudi-spark-datasource/pom.xml  | 2 +-
 hudi-sync/hudi-adb-sync/pom.xml| 2 +-
 hudi-sync/hudi-datahub-sync/pom.xml| 2 +-
 hudi-sync/hudi-hive-sync/pom.xml   | 2 +-
 hudi-sync/hudi-sync-common/pom.xml | 2 +-
 hudi-sync/pom.xml  | 2 +-
 hudi-tests-common/pom.xml  | 2 +-
 hudi-timeline-service/pom.xml  | 2 +-
 hudi-utilities/pom.xml | 2 +-
 packaging/hudi-aws-bundle/pom.xml  | 2 +-
 packaging/hudi-datahub-sync-bundle/pom.xml | 2 +-
 packaging/hudi-flink-bundle/pom.xml| 2 +-
 packaging/hudi-gcp-bundle/pom.xml  | 2 +-
 packaging/hudi-hadoop-mr-bundle/pom.xml| 2 +-
 packaging/hudi-hive-sync-bundle/pom.xml| 2 +-
 packaging/hudi-integ-test-bundle/pom.xml   | 2 +-
 packaging/hudi-kafka-connect-bundle/pom.xml| 2 +-
 packaging/hudi-presto-bundle/pom.xml   | 2 +-
 packaging/hudi-spark-bundle/pom.xml| 2 +-
 packaging/hudi-timeline-server-bundle/pom.xml  | 2 +-
 packaging/hudi-trino-bundle/pom.xml| 2 +-
 packaging/hudi-utilities-bundle/pom.xml| 2 +-
 packaging/hudi-utilities-slim-bundle/pom.xml   | 2 +-
 pom.xml| 

[GitHub] [hudi] hudi-bot commented on pull request #5416: [HUDI-3963] Use Lock-Free Message Queue Disruptor Improving Hoodie Writing Efficiency

2022-10-06 Thread GitBox


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

   
   ## CI report:
   
   * b838e1f406902c9bdfb5e84d53ef5a5effd0765b UNKNOWN
   * 6114ee2aa59f087e5ef0b1b53979eec143b33f5e UNKNOWN
   * 92760dbf5a047fe1f9941fa4b36c944eb3bec5c7 UNKNOWN
   * 3e44fb2397bdbfa9c63fab47e62f312451b91900 Azure: 
[SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12029)
 
   
   
   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-06 Thread GitBox


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

   
   ## CI report:
   
   * d99df9d68f351bd97aa8fee33cffc399f69efa9c Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12030)
 
   
   
   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-4990) Parallelize deduplication in CLI tool

2022-10-06 Thread Ethan Guo (Jira)


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

Ethan Guo updated HUDI-4990:

Description: The CLI tool command `repair deduplicate` repair one partition 
at a time.  To repair hundreds of partitions, this takes time.  We should add a 
mode to take multiple partition paths for the CLI and run the dedup job for 
multiple partitions at the same time.

> Parallelize deduplication in CLI tool
> -
>
> Key: HUDI-4990
> URL: https://issues.apache.org/jira/browse/HUDI-4990
> Project: Apache Hudi
>  Issue Type: Improvement
>Reporter: Ethan Guo
>Priority: Major
> Fix For: 0.13.0
>
>
> The CLI tool command `repair deduplicate` repair one partition at a time.  To 
> repair hundreds of partitions, this takes time.  We should add a mode to take 
> multiple partition paths for the CLI and run the dedup job for multiple 
> partitions at the same time.



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


[jira] [Created] (HUDI-4990) Parallelize deduplication in CLI tool

2022-10-06 Thread Ethan Guo (Jira)
Ethan Guo created HUDI-4990:
---

 Summary: Parallelize deduplication in CLI tool
 Key: HUDI-4990
 URL: https://issues.apache.org/jira/browse/HUDI-4990
 Project: Apache Hudi
  Issue Type: Improvement
Reporter: Ethan Guo






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


[jira] [Updated] (HUDI-4990) Parallelize deduplication in CLI tool

2022-10-06 Thread Ethan Guo (Jira)


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

Ethan Guo updated HUDI-4990:

Fix Version/s: 0.13.0

> Parallelize deduplication in CLI tool
> -
>
> Key: HUDI-4990
> URL: https://issues.apache.org/jira/browse/HUDI-4990
> 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)


[GitHub] [hudi] mikeashley87 commented on issue #6819: [SUPPORT] NotSerializeableException glue2.0 - spark 2.4.3, hudi-spark-bundle_2.11-0.11.0.jar

2022-10-06 Thread GitBox


mikeashley87 commented on issue #6819:
URL: https://github.com/apache/hudi/issues/6819#issuecomment-1270450572

   I am getting the same error when using
   
   
https://mvnrepository.com/artifact/org.apache.hudi/hudi-spark2.4-bundle_2.11/0.12.0
   
   
   ```
   py4j.protocol.Py4JJavaError: An error occurred while calling o1011.save.
   : org.apache.spark.SparkException: Job aborted due to stage failure: Failed 
to serialize task 3847, not attempting to retry it. Exception during 
serialization: java.io.NotSerializableException: org.apache.hadoop.fs.Path
   Serialization stack:
- object not serializable (class: org.apache.hadoop.fs.Path, value: 
s3://somebucket/someprefix)
- element of array (index: 0)
- array (class [Ljava.lang.Object;, size 1)
- field (class: scala.collection.mutable.WrappedArray$ofRef, name: 
array, type: class [Ljava.lang.Object;)
- object (class scala.collection.mutable.WrappedArray$ofRef, 
WrappedArray(s3://somebucket/someprefix))
- writeObject data (class: 
org.apache.spark.rdd.ParallelCollectionPartition)
- object (class org.apache.spark.rdd.ParallelCollectionPartition, 
org.apache.spark.rdd.ParallelCollectionPartition@3488)
- field (class: org.apache.spark.scheduler.ResultTask, name: partition, 
type: interface org.apache.spark.Partition)
- object (class org.apache.spark.scheduler.ResultTask, ResultTask(114, 
0))
at 
org.apache.spark.scheduler.DAGScheduler.org$apache$spark$scheduler$DAGScheduler$$failJobAndIndependentStages(DAGScheduler.scala:1889)
at 
org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1877)
at 
org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1876)
at 
scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48)
at 
org.apache.spark.scheduler.DAGScheduler.abortStage(DAGScheduler.scala:1876)
at 
org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:926)
at 
org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:926)
at scala.Option.foreach(Option.scala:257)
at 
org.apache.spark.scheduler.DAGScheduler.handleTaskSetFailed(DAGScheduler.scala:926)
at 
org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.doOnReceive(DAGScheduler.scala:2110)
at 
org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:2059)
at 
org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:2048)
at org.apache.spark.util.EventLoop$$anon$1.run(EventLoop.scala:49)
at 
org.apache.spark.scheduler.DAGScheduler.runJob(DAGScheduler.scala:737)
at org.apache.spark.SparkContext.runJob(SparkContext.scala:2061)
at org.apache.spark.SparkContext.runJob(SparkContext.scala:2082)
at org.apache.spark.SparkContext.runJob(SparkContext.scala:2101)
at org.apache.spark.SparkContext.runJob(SparkContext.scala:2126)
at org.apache.spark.rdd.RDD$$anonfun$collect$1.apply(RDD.scala:945)
at 
org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
at 
org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:112)
at org.apache.spark.rdd.RDD.withScope(RDD.scala:363)
at org.apache.spark.rdd.RDD.collect(RDD.scala:944)
at 
org.apache.spark.api.java.JavaRDDLike$class.collect(JavaRDDLike.scala:361)
at 
org.apache.spark.api.java.AbstractJavaRDDLike.collect(JavaRDDLike.scala:45)
at 
org.apache.hudi.client.common.HoodieSparkEngineContext.map(HoodieSparkEngineContext.java:103)
at 
org.apache.hudi.metadata.HoodieBackedTableMetadataWriter.listAllPartitions(HoodieBackedTableMetadataWriter.java:631)
at 
org.apache.hudi.metadata.HoodieBackedTableMetadataWriter.initialCommit(HoodieBackedTableMetadataWriter.java:1064)
at 
org.apache.hudi.metadata.HoodieBackedTableMetadataWriter.initializeFromFilesystem(HoodieBackedTableMetadataWriter.java:557)
at 
org.apache.hudi.metadata.HoodieBackedTableMetadataWriter.initializeIfNeeded(HoodieBackedTableMetadataWriter.java:390)
at 
org.apache.hudi.metadata.SparkHoodieBackedTableMetadataWriter.initialize(SparkHoodieBackedTableMetadataWriter.java:120)
at 
org.apache.hudi.metadata.HoodieBackedTableMetadataWriter.(HoodieBackedTableMetadataWriter.java:171)
at 
org.apache.hudi.metadata.SparkHoodieBackedTableMetadataWriter.(SparkHoodieBackedTableMetadataWriter.java:89)
at 
org.apache.hudi.metadata.SparkHoodieBackedTableMetadataWriter.create(SparkHoodieBackedTableMetadataWriter.java:75)
at 
org.apache.hudi.client.SparkRDDWriteClient.initializeMetadataTable(SparkRDDWriteClient.java:446)
at 

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

2022-10-06 Thread GitBox


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

   
   ## CI report:
   
   * 34e4db327b0d8394aa92126c97e833d75a0543b1 Azure: 
[CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12028)
 
   * d99df9d68f351bd97aa8fee33cffc399f69efa9c Azure: 
[PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12030)
 
   
   
   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-06 Thread GitBox


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

   
   ## CI report:
   
   *  Unknown: [CANCELED](TBD) 
   * 34e4db327b0d8394aa92126c97e833d75a0543b1 Azure: 
[PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12028)
 
   * d99df9d68f351bd97aa8fee33cffc399f69efa9c 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 #5416: [HUDI-3963] Use Lock-Free Message Queue Disruptor Improving Hoodie Writing Efficiency

2022-10-06 Thread GitBox


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

   
   ## CI report:
   
   * b838e1f406902c9bdfb5e84d53ef5a5effd0765b UNKNOWN
   * 6114ee2aa59f087e5ef0b1b53979eec143b33f5e UNKNOWN
   * 67d8cd3c14c8539a4033c9e4a99b16eef9a03c25 Azure: 
[SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12008)
 
   * 92760dbf5a047fe1f9941fa4b36c944eb3bec5c7 UNKNOWN
   * 3e44fb2397bdbfa9c63fab47e62f312451b91900 Azure: 
[PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12029)
 
   
   
   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 #6873: [HUDI-4971] Fix shading kryo-shaded with re-usable configs

2022-10-06 Thread GitBox


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

   
   ## CI report:
   
   * 83ef76c2bc6763f16ee342cc83435e907302f19f UNKNOWN
   * 9c309b8a1a9a843e60efb866361a8cf0bd077ba7 Azure: 
[SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12027)
 
   
   
   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] gtwuser commented on issue #6869: [SUPPORT] Incremental upsert or merge is not working

2022-10-06 Thread GitBox


gtwuser commented on issue #6869:
URL: https://github.com/apache/hudi/issues/6869#issuecomment-1270330139

   > ` 'hoodie.datasource.write.precombine.field': 'payload.recordedAt',
   > 
   > did you use a wrong precombine field?
   @fengjian428 sorry i recreated the payload for data privacy and missed to 
update the `recordKey`, updating the recordKey as per the given example


-- 
This is an automated message from the 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] the-other-tim-brown commented on a diff in pull request #6806: [HUDI-4905] Improve type handling in proto schema conversion

2022-10-06 Thread GitBox


the-other-tim-brown commented on code in PR #6806:
URL: https://github.com/apache/hudi/pull/6806#discussion_r989232387


##
hudi-utilities/pom.xml:
##
@@ -85,7 +85,6 @@
 
   com.google.protobuf
   protobuf-java-util
-  test

Review Comment:
   @alexeykudinkin and @codope 



-- 
This is an automated message from the 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-4690) Remove code duplicated over from Spark

2022-10-06 Thread Raymond Xu (Jira)


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

Raymond Xu updated HUDI-4690:
-
Status: In Progress  (was: Open)

> Remove code duplicated over from Spark
> --
>
> Key: HUDI-4690
> URL: https://issues.apache.org/jira/browse/HUDI-4690
> Project: Apache Hudi
>  Issue Type: Bug
>  Components: reader-core, writer-core
>Affects Versions: 0.12.0
>Reporter: Alexey Kudinkin
>Assignee: Alexey Kudinkin
>Priority: Blocker
>  Labels: pull-request-available
> Fix For: 0.13.0
>
>
> At present, a lot of code in `HoodieAnalysis` is unnecessarily duplicating 
> the resolution logic from Spark that leads to interference w/ normal 
> operations of Spark's Analyzer and leading to non-trivial issues (like 
> HUDI-4503) when dealing w/ Spark or Spark SQL
>  
> We should minimize the amount of logic and code that is localized from Spark 
> to Hudi to strictly necessary to either 
>  # Address issues (alternative to upstreaming in Spark)
>  # Back-port features (from newer Spark versions to older ones)
>  



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


[jira] [Updated] (HUDI-4503) Support table identifier with explicit catalog

2022-10-06 Thread Raymond Xu (Jira)


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

Raymond Xu updated HUDI-4503:
-
Status: In Progress  (was: Open)

> Support table identifier with explicit catalog
> --
>
> Key: HUDI-4503
> URL: https://issues.apache.org/jira/browse/HUDI-4503
> Project: Apache Hudi
>  Issue Type: Bug
>  Components: spark, spark-sql
>Reporter: Yann Byron
>Assignee: Alexey Kudinkin
>Priority: Blocker
>  Labels: pull-request-available
> Fix For: 0.13.0
>
>




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


[jira] [Updated] (HUDI-4503) Support table identifier with explicit catalog

2022-10-06 Thread Raymond Xu (Jira)


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

Raymond Xu updated HUDI-4503:
-
Status: Open  (was: In Progress)

> Support table identifier with explicit catalog
> --
>
> Key: HUDI-4503
> URL: https://issues.apache.org/jira/browse/HUDI-4503
> Project: Apache Hudi
>  Issue Type: Bug
>  Components: spark, spark-sql
>Reporter: Yann Byron
>Assignee: Alexey Kudinkin
>Priority: Blocker
>  Labels: pull-request-available
> Fix For: 0.13.0
>
>




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


[jira] [Updated] (HUDI-4690) Remove code duplicated over from Spark

2022-10-06 Thread Raymond Xu (Jira)


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

Raymond Xu updated HUDI-4690:
-
Status: Open  (was: In Progress)

> Remove code duplicated over from Spark
> --
>
> Key: HUDI-4690
> URL: https://issues.apache.org/jira/browse/HUDI-4690
> Project: Apache Hudi
>  Issue Type: Bug
>  Components: reader-core, writer-core
>Affects Versions: 0.12.0
>Reporter: Alexey Kudinkin
>Assignee: Alexey Kudinkin
>Priority: Blocker
>  Labels: pull-request-available
> Fix For: 0.13.0
>
>
> At present, a lot of code in `HoodieAnalysis` is unnecessarily duplicating 
> the resolution logic from Spark that leads to interference w/ normal 
> operations of Spark's Analyzer and leading to non-trivial issues (like 
> HUDI-4503) when dealing w/ Spark or Spark SQL
>  
> We should minimize the amount of logic and code that is localized from Spark 
> to Hudi to strictly necessary to either 
>  # Address issues (alternative to upstreaming in Spark)
>  # Back-port features (from newer Spark versions to older ones)
>  



--
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-06 Thread GitBox


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

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