From 527ade26e8098feb08d2488ce678b54c9006a4a0 Mon Sep 17 00:00:00 2001 From: yuezhang Date: Fri, 22 Apr 2022 16:49:58 +0800 Subject: [PATCH 01/50] need more test and tuning --- .../TestDisruptorExecutionInSpark.java | 161 +++++++++++++++ .../execution/TestDisruptorMessageQueue.java | 137 +++++++++++++ .../util/queue/BoundedInMemoryExecutor.java | 2 +- .../util/queue/BoundedInMemoryQueue.java | 4 +- .../queue/BoundedInMemoryQueueConsumer.java | 2 +- .../queue/BoundedInMemoryQueueProducer.java | 4 +- .../util/queue/DisruptorBasedProducer.java | 29 +++ .../common/util/queue/DisruptorExecutor.java | 187 ++++++++++++++++++ .../util/queue/DisruptorMessageHandler.java | 48 +++++ .../util/queue/DisruptorMessageQueue.java | 61 ++++++ .../common/util/queue/DisruptorPublisher.java | 36 ++++ .../queue/FunctionBasedDisruptorProducer.java | 44 +++++ .../queue/FunctionBasedQueueProducer.java | 2 +- .../common/util/queue/HoodieConsumer.java | 37 ++++ .../util/queue/HoodieDisruptorEvent.java | 32 +++ .../queue/HoodieDisruptorEventFactory.java | 29 +++ .../common/util/queue/HoodieExecutor.java | 34 ++++ .../common/util/queue/HoodieMessageQueue.java | 28 +++ .../common/util/queue/HoodieProducer.java | 22 +++ .../queue/IteratorBasedDisruptorProducer.java | 46 +++++ .../queue/IteratorBasedQueueProducer.java | 2 +- 21 files changed, 939 insertions(+), 8 deletions(-) create mode 100644 hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorExecutionInSpark.java create mode 100644 hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorMessageQueue.java create mode 100644 hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorBasedProducer.java create mode 100644 hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorExecutor.java create mode 100644 hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageHandler.java create mode 100644 hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageQueue.java create mode 100644 hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorPublisher.java create mode 100644 hudi-common/src/main/java/org/apache/hudi/common/util/queue/FunctionBasedDisruptorProducer.java create mode 100644 hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieConsumer.java create mode 100644 hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieDisruptorEvent.java create mode 100644 hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieDisruptorEventFactory.java create mode 100644 hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieExecutor.java create mode 100644 hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieMessageQueue.java create mode 100644 hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieProducer.java create mode 100644 hudi-common/src/main/java/org/apache/hudi/common/util/queue/IteratorBasedDisruptorProducer.java diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorExecutionInSpark.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorExecutionInSpark.java new file mode 100644 index 0000000000000..aa8f0128908ef --- /dev/null +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorExecutionInSpark.java @@ -0,0 +1,161 @@ +/* + * 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.execution; + +import static org.apache.hudi.execution.HoodieLazyInsertIterable.getTransformFunction; + +import org.apache.avro.generic.IndexedRecord; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; +import org.apache.hudi.common.testutils.HoodieTestDataGenerator; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.queue.BoundedInMemoryExecutor; +import org.apache.hudi.common.util.queue.BoundedInMemoryQueueConsumer; +import org.apache.hudi.common.util.queue.DisruptorExecutor; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.testutils.HoodieClientTestHarness; +import org.apache.spark.TaskContext; +import org.apache.spark.TaskContext$; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.util.List; + +import scala.Tuple2; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class TestDisruptorExecutionInSpark extends HoodieClientTestHarness { + + private final String instantTime = HoodieActiveTimeline.createNewInstantTime(); + + @BeforeEach + public void setUp() throws Exception { + initTestDataGenerator(); + initExecutorServiceWithFixedThreadPool(2); + } + + @AfterEach + public void tearDown() throws Exception { + cleanupResources(); + } + + private Runnable getPreExecuteRunnable() { + final TaskContext taskContext = TaskContext.get(); + return () -> TaskContext$.MODULE$.setTaskContext(taskContext); + } + + @Test + public void testExecutor() { + + final List hoodieRecords = dataGen.generateInserts(instantTime, 128); + + HoodieWriteConfig hoodieWriteConfig = mock(HoodieWriteConfig.class); + when(hoodieWriteConfig.getWriteBufferLimitBytes()).thenReturn(1024); + BoundedInMemoryQueueConsumer, Integer> consumer = + new BoundedInMemoryQueueConsumer, Integer>() { + + private int count = 0; + + @Override + protected void consumeOneRecord(HoodieLazyInsertIterable.HoodieInsertValueGenResult record) { + count++; + } + + @Override + protected void finish() { + } + + @Override + protected Integer getResult() { + return count; + } + }; + DisruptorExecutor>, Integer> exec = null; + + try { + exec = new DisruptorExecutor(hoodieWriteConfig.getWriteBufferLimitBytes(), hoodieRecords.iterator(), consumer, + getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA), getPreExecuteRunnable()); + int result = exec.execute(); + // It should buffer and write 100 records + assertEquals(128, result); + // There should be no remaining records in the buffer + assertFalse(exec.isRemaining()); + } finally { + if (exec != null) { + exec.shutdownNow(); + } + } + } + + @Test + public void testInterruptExecutor() { + final List hoodieRecords = dataGen.generateInserts(instantTime, 100); + + HoodieWriteConfig hoodieWriteConfig = mock(HoodieWriteConfig.class); + when(hoodieWriteConfig.getWriteBufferLimitBytes()).thenReturn(1024); + BoundedInMemoryQueueConsumer, Integer> consumer = + new BoundedInMemoryQueueConsumer, Integer>() { + + @Override + protected void consumeOneRecord(HoodieLazyInsertIterable.HoodieInsertValueGenResult record) { + try { + while (true) { + Thread.sleep(1000); + } + } catch (InterruptedException ie) { + return; + } + } + + @Override + protected void finish() { + } + + @Override + protected Integer getResult() { + return 0; + } + }; + + DisruptorExecutor>, Integer> executor = null; + try { + executor = new DisruptorExecutor(hoodieWriteConfig.getWriteBufferLimitBytes(), hoodieRecords.iterator(), consumer, + getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA), getPreExecuteRunnable()); + DisruptorExecutor>, Integer> finalExecutor = executor; + + Thread.currentThread().interrupt(); + + assertThrows(HoodieException.class, () -> finalExecutor.execute()); + assertTrue(Thread.interrupted()); + } finally { + if (executor != null) { + executor.shutdownNow(); + } + } + } +} diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorMessageQueue.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorMessageQueue.java new file mode 100644 index 0000000000000..0816d2894d034 --- /dev/null +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorMessageQueue.java @@ -0,0 +1,137 @@ +package org.apache.hudi.execution; + +import static org.apache.hudi.execution.HoodieLazyInsertIterable.getTransformFunction; + +import org.apache.avro.generic.IndexedRecord; +import org.apache.hudi.common.model.HoodieAvroRecord; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; +import org.apache.hudi.common.testutils.HoodieTestDataGenerator; +import org.apache.hudi.common.util.FileIOUtils; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.queue.BoundedInMemoryQueue; +import org.apache.hudi.common.util.queue.BoundedInMemoryQueueConsumer; +import org.apache.hudi.common.util.queue.DisruptorExecutor; +import org.apache.hudi.common.util.queue.DisruptorMessageQueue; +import org.apache.hudi.common.util.queue.IteratorBasedDisruptorProducer; +import org.apache.hudi.common.util.queue.IteratorBasedQueueProducer; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.testutils.HoodieClientTestHarness; +import org.apache.spark.TaskContext; +import org.apache.spark.TaskContext$; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.concurrent.Future; + +import scala.Tuple2; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class TestDisruptorMessageQueue extends HoodieClientTestHarness { + + private final String instantTime = HoodieActiveTimeline.createNewInstantTime(); + + @BeforeEach + public void setUp() throws Exception { + initTestDataGenerator(); + initExecutorServiceWithFixedThreadPool(2); + } + + @AfterEach + public void tearDown() throws Exception { + cleanupResources(); + } + + private Runnable getPreExecuteRunnable() { + final TaskContext taskContext = TaskContext.get(); + return () -> TaskContext$.MODULE$.setTaskContext(taskContext); + } + + // Test to ensure that we are reading all records from queue iterator in the same order + // without any exceptions. + @SuppressWarnings("unchecked") + @Test +// @Timeout(value = 60) + public void testRecordReading() throws Exception { + + final List hoodieRecords = dataGen.generateInserts(instantTime, 100); + ArrayList beforeRecord = new ArrayList<>(); + ArrayList beforeIndexedRecord = new ArrayList<>(); + ArrayList afterRecord = new ArrayList<>(); + ArrayList afterIndexedRecord = new ArrayList<>(); + + hoodieRecords.forEach(record -> { + final HoodieAvroRecord originalRecord = (HoodieAvroRecord) record; + beforeRecord.add(originalRecord); + try { + final Option originalInsertValue = + originalRecord.getData().getInsertValue(HoodieTestDataGenerator.AVRO_SCHEMA); + beforeIndexedRecord.add(originalInsertValue.get()); + } catch (IOException e) { + e.printStackTrace(); + } + }); + + HoodieWriteConfig hoodieWriteConfig = mock(HoodieWriteConfig.class); + when(hoodieWriteConfig.getWriteBufferLimitBytes()).thenReturn(1024); + BoundedInMemoryQueueConsumer, Integer> consumer = + new BoundedInMemoryQueueConsumer, Integer>() { + + private int count = 0; + + @Override + protected void consumeOneRecord(HoodieLazyInsertIterable.HoodieInsertValueGenResult record) { + count++; + afterRecord.add((HoodieAvroRecord) record.record); + try { + IndexedRecord indexedRecord = (IndexedRecord)((HoodieAvroRecord) record.record) + .getData().getInsertValue(HoodieTestDataGenerator.AVRO_SCHEMA).get(); + afterIndexedRecord.add(indexedRecord); + } catch (IOException e) { + e.printStackTrace(); + } + } + + @Override + protected void finish() { + } + + @Override + protected Integer getResult() { + return count; + } + }; + + DisruptorExecutor>, Integer> exec = null; + + try { + exec = new DisruptorExecutor(hoodieWriteConfig.getWriteBufferLimitBytes(), hoodieRecords.iterator(), consumer, + getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA), getPreExecuteRunnable()); + int result = exec.execute(); + // It should buffer and write 100 records + assertEquals(100, result); + // There should be no remaining records in the buffer + assertFalse(exec.isRemaining()); + + assertEquals(beforeRecord, afterRecord); + assertEquals(beforeIndexedRecord, afterIndexedRecord); + System.out.println(); + + } finally { + if (exec != null) { + exec.shutdownNow(); + } + } + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryExecutor.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryExecutor.java index d1e5e66083196..4fe228b75c533 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryExecutor.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryExecutor.java @@ -45,7 +45,7 @@ * class takes as input the size limit, queue producer(s), consumer and transformer and exposes API to orchestrate * concurrent execution of these actors communicating through a central bounded queue */ -public class BoundedInMemoryExecutor { +public class BoundedInMemoryExecutor extends HoodieExecutor { private static final Logger LOG = LogManager.getLogger(BoundedInMemoryExecutor.class); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryQueue.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryQueue.java index dfe33b49ec0c7..4727d0d9b3f01 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryQueue.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryQueue.java @@ -49,7 +49,7 @@ * @param input payload data type * @param output payload data type */ -public class BoundedInMemoryQueue implements Iterable { +public class BoundedInMemoryQueue extends HoodieMessageQueue implements Iterable { /** Interval used for polling records in the queue. **/ public static final int RECORD_POLL_INTERVAL_SEC = 1; @@ -135,7 +135,7 @@ public BoundedInMemoryQueue(final long memoryLimit, final Function transfo this.iterator = new QueueIterator(); } - public int size() { + public long size() { return this.queue.size(); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryQueueConsumer.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryQueueConsumer.java index c34842fbe3d01..e8e77e3213d2f 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryQueueConsumer.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryQueueConsumer.java @@ -23,7 +23,7 @@ /** * Consume entries from queue and execute callback function. */ -public abstract class BoundedInMemoryQueueConsumer { +public abstract class BoundedInMemoryQueueConsumer extends HoodieConsumer { /** * API to de-queue entries to memory bounded queue. diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryQueueProducer.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryQueueProducer.java index ecea9f2193c76..945416de169da 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryQueueProducer.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryQueueProducer.java @@ -23,12 +23,12 @@ * * @param Input type for buffer items produced */ -public interface BoundedInMemoryQueueProducer { +public abstract class BoundedInMemoryQueueProducer implements HoodieProducer { /** * API to enqueue entries to memory bounded queue. * * @param queue In Memory bounded queue */ - void produce(BoundedInMemoryQueue queue) throws Exception; + public abstract void produce(BoundedInMemoryQueue queue) throws Exception; } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorBasedProducer.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorBasedProducer.java new file mode 100644 index 0000000000000..9f7eb7d5d2e10 --- /dev/null +++ b/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 { + + /** + * API to enqueue entries to memory bounded queue. + * + * @param queue In Memory bounded queue + */ + public abstract void produce(DisruptorMessageQueue queue) throws Exception; +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorExecutor.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorExecutor.java new file mode 100644 index 0000000000000..6f806eed5ed72 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorExecutor.java @@ -0,0 +1,187 @@ +/* + * 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.DefaultSizeEstimator; +import org.apache.hudi.common.util.Functions; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.SizeEstimator; +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.CompletableFuture; +import java.util.concurrent.ExecutorCompletionService; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.concurrent.atomic.AtomicBoolean; +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 long bufferLimitInBytes, final Iterator inputItr, + BoundedInMemoryQueueConsumer consumer, Function transformFunction, Runnable preExecuteRunnable) { + this(bufferLimitInBytes, new IteratorBasedDisruptorProducer<>(inputItr), Option.of(consumer), transformFunction, preExecuteRunnable); + } + + public DisruptorExecutor(final long bufferLimitInBytes, DisruptorBasedProducer producer, + Option> consumer, final Function transformFunction) { + this(bufferLimitInBytes, producer, consumer, transformFunction, Functions.noop()); + } + + public DisruptorExecutor(final long bufferLimitInBytes, DisruptorBasedProducer producer, + Option> consumer, final Function transformFunction, Runnable preExecuteRunnable) { + this(bufferLimitInBytes, Collections.singletonList(producer), consumer, transformFunction, new DefaultSizeEstimator<>(), preExecuteRunnable); + } + + public DisruptorExecutor(final long bufferLimitInBytes, List> producers, + Option> consumer, final Function transformFunction, + final SizeEstimator sizeEstimator, 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<>(bufferLimitInBytes, transformFunction, sizeEstimator, producers.size()); + } + + + /** + * 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 e; + } + return true; + }); + }).collect(Collectors.toList()); + return completionService; + } + + @Override + public E execute() { + try { + Future future = startConsumer(queue.getInnerQueue()); + startProducers(); + // Wait for consumer to be done + return future.get(); + } catch (InterruptedException ie) { + shutdownNow(); + Thread.currentThread().interrupt(); + throw new HoodieException(ie); + } catch (Exception e) { + throw new HoodieException(e); + } + } + + /** + * Start only consumer. + */ + private Future startConsumer(Disruptor> disruptor) { + AtomicBoolean isRegister = new AtomicBoolean(false); + Future future = consumer.map(consumer -> { + return consumerExecutorService.submit(() -> { + LOG.info("starting consumer thread"); + preExecuteRunnable.run(); + try { + DisruptorMessageHandler handler = new DisruptorMessageHandler<>(consumer, preExecuteRunnable); + disruptor.handleEventsWith(handler); + + // start disruptor + queue.getInnerQueue().start(); + isRegister.set(true); + while (!handler.isFinished()) { + Thread.sleep(1 * 1000); + } + + LOG.info("Queue Consumption is done; notifying producer threads"); + consumer.finish(); + return consumer.getResult(); + } catch (Exception e) { + LOG.error("error consuming records", e); + throw e; + } + }); + }).orElse(CompletableFuture.completedFuture(null)); + + // waiting until consumer registered. + while (!isRegister.get()) { + try { + Thread.sleep(1 * 1000); + } catch (InterruptedException e) { + // ignore here + } + } + + return future; + } + + @Override + public boolean isRemaining() { + return false; + } + + @Override + public void shutdownNow() { + producerExecutorService.shutdownNow(); + consumerExecutorService.shutdownNow(); + queue.close(); + } + + @Override + public DisruptorMessageQueue getQueue() { + return queue; + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageHandler.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageHandler.java new file mode 100644 index 0000000000000..628dd255b1390 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageHandler.java @@ -0,0 +1,48 @@ +/* + * 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.EventHandler; + +public class DisruptorMessageHandler implements EventHandler> { + + private BoundedInMemoryQueueConsumer consumer; + private boolean finished = false; + private Runnable preExecuteRunnable; + + public DisruptorMessageHandler(BoundedInMemoryQueueConsumer consumer, Runnable preExecuteRunnable) { + this.consumer = consumer; + this.preExecuteRunnable = preExecuteRunnable; + } + + @Override + public void onEvent(HoodieDisruptorEvent event, long sequence, boolean endOfBatch) { + preExecuteRunnable.run(); + if (event == null || event.get() == null) { + // end of ingestion + finished = true; + } else { + consumer.consumeOneRecord(event.get()); + } + } + + public boolean isFinished() { + return finished; + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageQueue.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageQueue.java new file mode 100644 index 0000000000000..97461d11e9eb2 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageQueue.java @@ -0,0 +1,61 @@ +package org.apache.hudi.common.util.queue; + +import com.lmax.disruptor.BlockingWaitStrategy; + +import com.lmax.disruptor.EventTranslator; +import com.lmax.disruptor.dsl.Disruptor; +import com.lmax.disruptor.dsl.ProducerType; +import com.lmax.disruptor.util.DaemonThreadFactory; +import org.apache.hudi.common.util.SizeEstimator; + +import java.util.function.Function; + +public class DisruptorMessageQueue extends HoodieMessageQueue { + + private final Disruptor> queue; + private final Function transformFunction; + + private final int bufferSize = 1024; + + public DisruptorMessageQueue(long bufferLimitInBytes, Function transformFunction, SizeEstimator sizeEstimator, int producerNumber) { + if (producerNumber > 1) { + this.queue = new Disruptor<>(HoodieDisruptorEvent::new, bufferSize, DaemonThreadFactory.INSTANCE, ProducerType.MULTI, new BlockingWaitStrategy()); + } else { + this.queue = new Disruptor<>(HoodieDisruptorEvent::new, bufferSize, DaemonThreadFactory.INSTANCE, ProducerType.SINGLE, new BlockingWaitStrategy()); + } + this.transformFunction = transformFunction; + } + + @Override + public long size() { + return queue.getBufferSize(); + } + + @Override + public void insertRecord(I value) throws Exception { + O applied; + if (value == null) { + applied = null; + } else { + 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() { + return this.queue; + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorPublisher.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorPublisher.java new file mode 100644 index 0000000000000..fdfc3e44d5d8b --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorPublisher.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 com.lmax.disruptor.RingBuffer; + +public class DisruptorPublisher { + + DisruptorBasedProducer producer; + private final DisruptorMessageQueue queue; + + public DisruptorPublisher(DisruptorBasedProducer producer, DisruptorMessageQueue queue) { + this.producer = producer; + this.queue = queue; + } + + public void startProduce() throws Exception { + producer.produce(queue); + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/FunctionBasedDisruptorProducer.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/FunctionBasedDisruptorProducer.java new file mode 100644 index 0000000000000..8aa8c6da8eeee --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/FunctionBasedDisruptorProducer.java @@ -0,0 +1,44 @@ +/* + * 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.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import java.util.function.Function; + +public class FunctionBasedDisruptorProducer extends DisruptorBasedProducer { + + private static final Logger LOG = LogManager.getLogger(FunctionBasedDisruptorProducer.class); + + private final Function, Boolean> producerFunction; + + public FunctionBasedDisruptorProducer(Function, Boolean> producerFunction) { + this.producerFunction = producerFunction; + } + + @Override + public void produce(DisruptorMessageQueue queue) throws Exception { + LOG.info("starting function which will enqueue records"); + producerFunction.apply(queue); + LOG.info("finished function which will enqueue records"); + // poison after buffer finished. + queue.insertRecord(null); + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/FunctionBasedQueueProducer.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/FunctionBasedQueueProducer.java index 549683754cbd8..d391c0e8c6192 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/FunctionBasedQueueProducer.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/FunctionBasedQueueProducer.java @@ -28,7 +28,7 @@ * * @param Type of entry produced for queue */ -public class FunctionBasedQueueProducer implements BoundedInMemoryQueueProducer { +public class FunctionBasedQueueProducer extends BoundedInMemoryQueueProducer { private static final Logger LOG = LogManager.getLogger(FunctionBasedQueueProducer.class); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieConsumer.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieConsumer.java new file mode 100644 index 0000000000000..e308f41c8da78 --- /dev/null +++ b/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(); +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieDisruptorEvent.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieDisruptorEvent.java new file mode 100644 index 0000000000000..741245a3513e6 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieDisruptorEvent.java @@ -0,0 +1,32 @@ +/* + * 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 { + + private O value; + + public void set(O value) { + this.value = value; + } + + public O get() { + return this.value; + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieDisruptorEventFactory.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieDisruptorEventFactory.java new file mode 100644 index 0000000000000..76c8a4c2e5a15 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieDisruptorEventFactory.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; + +import com.lmax.disruptor.EventFactory; + +public class HoodieDisruptorEventFactory implements EventFactory { + + @Override + public HoodieDisruptorEvent newInstance() { + return new HoodieDisruptorEvent(); + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieExecutor.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieExecutor.java new file mode 100644 index 0000000000000..4a4106bef0200 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieExecutor.java @@ -0,0 +1,34 @@ +/* + * 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 { + + public abstract ExecutorCompletionService startProducers(); + + public abstract E execute(); + + public abstract boolean isRemaining(); + + public abstract void shutdownNow(); + + public abstract HoodieMessageQueue getQueue(); +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieMessageQueue.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieMessageQueue.java new file mode 100644 index 0000000000000..fe7848b1e09bc --- /dev/null +++ b/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 { + + public abstract long size(); + + public abstract void insertRecord(I t) throws Exception; + + public abstract void close(); +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieProducer.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieProducer.java new file mode 100644 index 0000000000000..736f9b93fadbd --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieProducer.java @@ -0,0 +1,22 @@ +/* + * 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 interface HoodieProducer { +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/IteratorBasedDisruptorProducer.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/IteratorBasedDisruptorProducer.java new file mode 100644 index 0000000000000..06bfc0c9bd862 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/IteratorBasedDisruptorProducer.java @@ -0,0 +1,46 @@ +/* + * 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.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import java.util.Iterator; + +public class IteratorBasedDisruptorProducer extends DisruptorBasedProducer { + + private static final Logger LOG = LogManager.getLogger(IteratorBasedDisruptorProducer.class); + + // input iterator for producing items in the buffer. + private final Iterator inputIterator; + + public IteratorBasedDisruptorProducer(Iterator inputIterator) { + this.inputIterator = inputIterator; + } + @Override + public void produce(DisruptorMessageQueue queue) throws Exception { + LOG.info("starting to buffer records"); + while (inputIterator.hasNext()) { + queue.insertRecord(inputIterator.next()); + } + // poison after buffer finished. + queue.insertRecord(null); + LOG.info("finished buffering records"); + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/IteratorBasedQueueProducer.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/IteratorBasedQueueProducer.java index 3d11f38e5cf50..8459e7770d59e 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/IteratorBasedQueueProducer.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/IteratorBasedQueueProducer.java @@ -28,7 +28,7 @@ * * @param Item type produced for the buffer. */ -public class IteratorBasedQueueProducer implements BoundedInMemoryQueueProducer { +public class IteratorBasedQueueProducer extends BoundedInMemoryQueueProducer { private static final Logger LOG = LogManager.getLogger(IteratorBasedQueueProducer.class); From 2b096e86255d0447028b894b2730953a65254096 Mon Sep 17 00:00:00 2001 From: yuezhang Date: Fri, 22 Apr 2022 18:52:09 +0800 Subject: [PATCH 02/50] checked code style&& still need more java docs and more config and enable if necessary --- .../execution/SparkLazyInsertIterable.java | 9 +++++-- .../TestDisruptorExecutionInSpark.java | 26 ++++++++++++++----- .../execution/TestDisruptorMessageQueue.java | 11 +------- hudi-common/pom.xml | 7 +++++ .../common/util/queue/DisruptorExecutor.java | 3 ++- .../util/queue/DisruptorMessageHandler.java | 1 + .../util/queue/DisruptorMessageQueue.java | 20 +++++++++----- .../common/util/queue/DisruptorPublisher.java | 2 -- .../util/queue/HoodieDisruptorEvent.java | 4 +++ .../queue/IteratorBasedDisruptorProducer.java | 1 + 10 files changed, 57 insertions(+), 27 deletions(-) diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/SparkLazyInsertIterable.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/SparkLazyInsertIterable.java index a8a9e49c01c00..7908ead8f98d3 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/SparkLazyInsertIterable.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/SparkLazyInsertIterable.java @@ -24,6 +24,7 @@ import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.util.queue.BoundedInMemoryExecutor; +import org.apache.hudi.common.util.queue.DisruptorExecutor; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.io.WriteHandleFactory; @@ -77,15 +78,19 @@ public SparkLazyInsertIterable(Iterator> recordItr, @Override protected List computeNext() { // Executor service used for launching writer thread. - BoundedInMemoryExecutor, HoodieInsertValueGenResult, List> bufferedIteratorExecutor = + DisruptorExecutor, HoodieInsertValueGenResult, List> bufferedIteratorExecutor = null; try { Schema schema = new Schema.Parser().parse(hoodieConfig.getSchema()); if (useWriterSchema) { schema = HoodieAvroUtils.addMetadataFields(schema); } +// bufferedIteratorExecutor = +// new BoundedInMemoryExecutor<>(hoodieConfig.getWriteBufferLimitBytes(), inputItr, getInsertHandler(), +// getTransformFunction(schema, hoodieConfig), hoodieTable.getPreExecuteRunnable()); + bufferedIteratorExecutor = - new BoundedInMemoryExecutor<>(hoodieConfig.getWriteBufferLimitBytes(), inputItr, getInsertHandler(), + new DisruptorExecutor<>(hoodieConfig.getWriteBufferLimitBytes(), inputItr, getInsertHandler(), getTransformFunction(schema, hoodieConfig), hoodieTable.getPreExecuteRunnable()); final List result = bufferedIteratorExecutor.execute(); assert result != null && !result.isEmpty() && !bufferedIteratorExecutor.isRemaining(); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorExecutionInSpark.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorExecutionInSpark.java index aa8f0128908ef..72afac83ad0cc 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorExecutionInSpark.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorExecutionInSpark.java @@ -26,7 +26,6 @@ import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.common.util.Option; -import org.apache.hudi.common.util.queue.BoundedInMemoryExecutor; import org.apache.hudi.common.util.queue.BoundedInMemoryQueueConsumer; import org.apache.hudi.common.util.queue.DisruptorExecutor; import org.apache.hudi.config.HoodieWriteConfig; @@ -39,12 +38,15 @@ import org.junit.jupiter.api.Test; import java.util.List; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.concurrent.atomic.AtomicReference; import scala.Tuple2; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; -import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -115,6 +117,7 @@ protected Integer getResult() { @Test public void testInterruptExecutor() { final List hoodieRecords = dataGen.generateInserts(instantTime, 100); + ExecutorService pool = Executors.newSingleThreadExecutor(); HoodieWriteConfig hoodieWriteConfig = mock(HoodieWriteConfig.class); when(hoodieWriteConfig.getWriteBufferLimitBytes()).thenReturn(1024); @@ -143,15 +146,26 @@ protected Integer getResult() { }; DisruptorExecutor>, Integer> executor = null; + AtomicReference actualException = new AtomicReference<>(); try { executor = new DisruptorExecutor(hoodieWriteConfig.getWriteBufferLimitBytes(), hoodieRecords.iterator(), consumer, getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA), getPreExecuteRunnable()); DisruptorExecutor>, Integer> finalExecutor = executor; - Thread.currentThread().interrupt(); - - assertThrows(HoodieException.class, () -> finalExecutor.execute()); - assertTrue(Thread.interrupted()); + Future future = pool.submit(() -> { + try { + finalExecutor.execute(); + } catch (Exception e) { + actualException.set(e); + } + + }); + Thread.sleep(2 * 1000); + future.cancel(true); + future.get(); + assertTrue(actualException.get() instanceof HoodieException); + } catch (Exception e) { + // ignore here } finally { if (executor != null) { executor.shutdownNow(); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorMessageQueue.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorMessageQueue.java index 0816d2894d034..487b67c966468 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorMessageQueue.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorMessageQueue.java @@ -7,14 +7,9 @@ import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; -import org.apache.hudi.common.util.FileIOUtils; import org.apache.hudi.common.util.Option; -import org.apache.hudi.common.util.queue.BoundedInMemoryQueue; import org.apache.hudi.common.util.queue.BoundedInMemoryQueueConsumer; import org.apache.hudi.common.util.queue.DisruptorExecutor; -import org.apache.hudi.common.util.queue.DisruptorMessageQueue; -import org.apache.hudi.common.util.queue.IteratorBasedDisruptorProducer; -import org.apache.hudi.common.util.queue.IteratorBasedQueueProducer; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.testutils.HoodieClientTestHarness; import org.apache.spark.TaskContext; @@ -26,15 +21,12 @@ import java.io.IOException; import java.util.ArrayList; -import java.util.Iterator; import java.util.List; -import java.util.concurrent.Future; import scala.Tuple2; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; -import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -62,7 +54,7 @@ private Runnable getPreExecuteRunnable() { // without any exceptions. @SuppressWarnings("unchecked") @Test -// @Timeout(value = 60) + @Timeout(value = 60) public void testRecordReading() throws Exception { final List hoodieRecords = dataGen.generateInserts(instantTime, 100); @@ -126,7 +118,6 @@ protected Integer getResult() { assertEquals(beforeRecord, afterRecord); assertEquals(beforeIndexedRecord, afterIndexedRecord); - System.out.println(); } finally { if (exec != null) { diff --git a/hudi-common/pom.xml b/hudi-common/pom.xml index 251889c17fcc4..962807b0c852d 100644 --- a/hudi-common/pom.xml +++ b/hudi-common/pom.xml @@ -271,5 +271,12 @@ test + + com.lmax + disruptor + 3.4.2 + + + diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorExecutor.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorExecutor.java index 6f806eed5ed72..cb22b4c8461a4 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorExecutor.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorExecutor.java @@ -25,6 +25,7 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.SizeEstimator; import org.apache.hudi.exception.HoodieException; + import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -177,7 +178,7 @@ public boolean isRemaining() { public void shutdownNow() { producerExecutorService.shutdownNow(); consumerExecutorService.shutdownNow(); - queue.close(); + queue.shutdownNow(); } @Override diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageHandler.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageHandler.java index 628dd255b1390..cbaf70b50b705 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageHandler.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageHandler.java @@ -39,6 +39,7 @@ public void onEvent(HoodieDisruptorEvent event, long sequence, boolean endOfB finished = true; } else { consumer.consumeOneRecord(event.get()); + event.clear(); } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageQueue.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageQueue.java index 97461d11e9eb2..667cc09a52a34 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageQueue.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageQueue.java @@ -1,28 +1,32 @@ package org.apache.hudi.common.util.queue; -import com.lmax.disruptor.BlockingWaitStrategy; - +import com.lmax.disruptor.BusySpinWaitStrategy; import com.lmax.disruptor.EventTranslator; import com.lmax.disruptor.dsl.Disruptor; import com.lmax.disruptor.dsl.ProducerType; -import com.lmax.disruptor.util.DaemonThreadFactory; import org.apache.hudi.common.util.SizeEstimator; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; import java.util.function.Function; public class DisruptorMessageQueue extends HoodieMessageQueue { private final Disruptor> queue; private final Function transformFunction; + private ExecutorService executorService; - private final int bufferSize = 1024; + private final int bufferSize = 128 * 1024; public DisruptorMessageQueue(long bufferLimitInBytes, Function transformFunction, SizeEstimator sizeEstimator, int producerNumber) { + this.executorService = Executors.newCachedThreadPool(); + if (producerNumber > 1) { - this.queue = new Disruptor<>(HoodieDisruptorEvent::new, bufferSize, DaemonThreadFactory.INSTANCE, ProducerType.MULTI, new BlockingWaitStrategy()); + this.queue = new Disruptor<>(HoodieDisruptorEvent::new, bufferSize, executorService, ProducerType.MULTI, new BusySpinWaitStrategy()); } else { - this.queue = new Disruptor<>(HoodieDisruptorEvent::new, bufferSize, DaemonThreadFactory.INSTANCE, ProducerType.SINGLE, new BlockingWaitStrategy()); + this.queue = new Disruptor<>(HoodieDisruptorEvent::new, bufferSize, executorService, ProducerType.SINGLE, new BusySpinWaitStrategy()); } + this.transformFunction = transformFunction; } @@ -55,6 +59,10 @@ public void close() { queue.shutdown(); } + public void shutdownNow() { + executorService.shutdownNow(); + } + public Disruptor> getInnerQueue() { return this.queue; } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorPublisher.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorPublisher.java index fdfc3e44d5d8b..fe3378fa1d487 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorPublisher.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorPublisher.java @@ -18,8 +18,6 @@ package org.apache.hudi.common.util.queue; -import com.lmax.disruptor.RingBuffer; - public class DisruptorPublisher { DisruptorBasedProducer producer; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieDisruptorEvent.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieDisruptorEvent.java index 741245a3513e6..4f64132bef765 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieDisruptorEvent.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieDisruptorEvent.java @@ -29,4 +29,8 @@ public void set(O value) { public O get() { return this.value; } + + public void clear() { + value = null; + } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/IteratorBasedDisruptorProducer.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/IteratorBasedDisruptorProducer.java index 06bfc0c9bd862..1f1c68ea9c0c5 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/IteratorBasedDisruptorProducer.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/IteratorBasedDisruptorProducer.java @@ -33,6 +33,7 @@ public class IteratorBasedDisruptorProducer extends DisruptorBasedProducer public IteratorBasedDisruptorProducer(Iterator inputIterator) { this.inputIterator = inputIterator; } + @Override public void produce(DisruptorMessageQueue queue) throws Exception { LOG.info("starting to buffer records"); From 19ce784192eb0d03a7fd7713ed85d16fc275f56e Mon Sep 17 00:00:00 2001 From: yuezhang Date: Sat, 23 Apr 2022 09:07:43 +0800 Subject: [PATCH 03/50] config and need more test --- .../apache/hudi/config/HoodieWriteConfig.java | 45 +++++++++++++++++ .../execution/SparkLazyInsertIterable.java | 40 +++++++++++---- .../TestDisruptorExecutionInSpark.java | 5 +- .../execution/TestDisruptorMessageQueue.java | 3 +- .../common/util/queue/DisruptorExecutor.java | 24 ++++----- .../util/queue/DisruptorMessageQueue.java | 19 ++----- .../hudi/common/util/queue/ExecutorType.java | 49 ++++++++++++++++++ .../util/queue/HoodieDaemonThreadFactory.java | 35 +++++++++++++ .../queue/HoodieDisruptorEventFactory.java | 6 +-- .../util/queue/WaitStrategyFactory.java | 50 +++++++++++++++++++ 10 files changed, 233 insertions(+), 43 deletions(-) create mode 100644 hudi-common/src/main/java/org/apache/hudi/common/util/queue/ExecutorType.java create mode 100644 hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieDaemonThreadFactory.java create mode 100644 hudi-common/src/main/java/org/apache/hudi/common/util/queue/WaitStrategyFactory.java diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java index ca201816cb045..d0b2787f269b2 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java @@ -127,6 +127,14 @@ public class HoodieWriteConfig extends HoodieConfig { .withDocumentation("Key generator class, that implements `org.apache.hudi.keygen.KeyGenerator` " + "extract a key out of incoming records."); + public static final ConfigProperty EXECUTOR_TYPE = ConfigProperty + .key("hoodie.write.executor.type") + .defaultValue("BOUNDED_IN_MEMORY_EXECUTOR") + .withDocumentation("Set executor which orchestrates concurrent producers and consumers communicating through a message queue." + + "default value is BOUNDED_IN_MEMORY_EXECUTOR which use a bounded in-memory queue using LinkedBlockingQueue." + + "Also users could use DISRUPTOR_EXECUTOR, which use disruptor as a lock free message queue " + + "to gain better writing performance. Although DISRUPTOR_EXECUTOR is still an experimental feature."); + public static final ConfigProperty KEYGENERATOR_TYPE = ConfigProperty .key("hoodie.datasource.write.keygenerator.type") .defaultValue(KeyGeneratorType.SIMPLE.name()) @@ -233,6 +241,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(128 * 1024) + .withDocumentation("The size of the Disruptor Executor ring buffer, must be power of 2"); + + public static final ConfigProperty WRITE_WAIT_STRATEGY = ConfigProperty + .key("hoodie.write.wait.strategy") + .defaultValue("BlockingWaitStrategy") + .withDocumentation("Strategy employed for making DisruptorExecutor wait on a cursor."); + public static final ConfigProperty COMBINE_BEFORE_INSERT = ConfigProperty .key("hoodie.combine.before.insert") .defaultValue("false") @@ -971,6 +989,10 @@ public String getKeyGeneratorClass() { return getString(KEYGENERATOR_CLASS_NAME); } + public String getExecutorType() { + return getString(EXECUTOR_TYPE); + } + public boolean isConsistentLogicalTimestampEnabled() { return getBooleanOrDefault(KeyGeneratorOptions.KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED); } @@ -1031,6 +1053,14 @@ public int getWriteBufferLimitBytes() { return Integer.parseInt(getStringOrDefault(WRITE_BUFFER_LIMIT_BYTES_VALUE)); } + public String getWriteWaitStrategy() { + return getString(WRITE_WAIT_STRATEGY); + } + + public int getWriteBufferSize() { + return getInt(WRITE_BUFFER_SIZE); + } + public boolean shouldCombineBeforeInsert() { return getBoolean(COMBINE_BEFORE_INSERT); } @@ -2170,6 +2200,11 @@ public Builder withKeyGenerator(String keyGeneratorClass) { return this; } + public Builder withExecutorName(String executorClass) { + writeConfig.setValue(EXECUTOR_TYPE, executorClass); + return this; + } + public Builder withTimelineLayoutVersion(int version) { writeConfig.setValue(TIMELINE_LAYOUT_VERSION_NUM, String.valueOf(version)); return this; @@ -2216,6 +2251,16 @@ public Builder withWriteBufferLimitBytes(int writeBufferLimit) { return this; } + public Builder withWriteWaitStrategy(String waitStrategy) { + writeConfig.setValue(WRITE_WAIT_STRATEGY, String.valueOf(waitStrategy)); + return this; + } + + public Builder withWriteBufferSize(int size) { + writeConfig.setValue(WRITE_BUFFER_SIZE, String.valueOf(size)); + return this; + } + public Builder combineInput(boolean onInsert, boolean onUpsert) { writeConfig.setValue(COMBINE_BEFORE_INSERT, String.valueOf(onInsert)); writeConfig.setValue(COMBINE_BEFORE_UPSERT, String.valueOf(onUpsert)); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/SparkLazyInsertIterable.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/SparkLazyInsertIterable.java index 7908ead8f98d3..af51df04d8f86 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/SparkLazyInsertIterable.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/SparkLazyInsertIterable.java @@ -25,6 +25,8 @@ import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.util.queue.BoundedInMemoryExecutor; import org.apache.hudi.common.util.queue.DisruptorExecutor; +import org.apache.hudi.common.util.queue.ExecutorType; +import org.apache.hudi.common.util.queue.HoodieExecutor; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.io.WriteHandleFactory; @@ -34,6 +36,7 @@ import java.util.Iterator; import java.util.List; +import java.util.Locale; public class SparkLazyInsertIterable extends HoodieLazyInsertIterable { @@ -78,28 +81,43 @@ public SparkLazyInsertIterable(Iterator> recordItr, @Override protected List computeNext() { // Executor service used for launching writer thread. - DisruptorExecutor, HoodieInsertValueGenResult, List> bufferedIteratorExecutor = - null; + HoodieExecutor> executor = null; try { Schema schema = new Schema.Parser().parse(hoodieConfig.getSchema()); if (useWriterSchema) { schema = HoodieAvroUtils.addMetadataFields(schema); } -// bufferedIteratorExecutor = -// new BoundedInMemoryExecutor<>(hoodieConfig.getWriteBufferLimitBytes(), inputItr, getInsertHandler(), -// getTransformFunction(schema, hoodieConfig), hoodieTable.getPreExecuteRunnable()); - bufferedIteratorExecutor = - new DisruptorExecutor<>(hoodieConfig.getWriteBufferLimitBytes(), inputItr, getInsertHandler(), + String executorType = hoodieConfig.getExecutorType(); + ExecutorType executorTypeEnum; + + try { + executorTypeEnum = ExecutorType.valueOf(executorType.toUpperCase(Locale.ROOT)); + } catch (IllegalArgumentException e) { + throw new HoodieException("Unsupported Executor Type " + executorType); + } + + switch (executorTypeEnum) { + case BOUNDED_IN_MEMORY_EXECUTOR: + executor = new BoundedInMemoryExecutor<>(hoodieConfig.getWriteBufferLimitBytes(), inputItr, getInsertHandler(), getTransformFunction(schema, hoodieConfig), hoodieTable.getPreExecuteRunnable()); - final List result = bufferedIteratorExecutor.execute(); - assert result != null && !result.isEmpty() && !bufferedIteratorExecutor.isRemaining(); + break; + case DISRUPTOR_EXECUTOR: + executor = new DisruptorExecutor<>(hoodieConfig.getWriteBufferSize(), inputItr, getInsertHandler(), + getTransformFunction(schema, hoodieConfig), hoodieConfig.getWriteWaitStrategy(), hoodieTable.getPreExecuteRunnable()); + break; + default: + throw new HoodieException("Unsupported Executor Type " + executorType); + } + + final List result = executor.execute(); + assert result != null && !result.isEmpty() && !executor.isRemaining(); return result; } catch (Exception e) { throw new HoodieException(e); } finally { - if (null != bufferedIteratorExecutor) { - bufferedIteratorExecutor.shutdownNow(); + if (null != executor) { + executor.shutdownNow(); } } } diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorExecutionInSpark.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorExecutionInSpark.java index 72afac83ad0cc..529fdffe1498c 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorExecutionInSpark.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorExecutionInSpark.java @@ -28,6 +28,7 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.queue.BoundedInMemoryQueueConsumer; import org.apache.hudi.common.util.queue.DisruptorExecutor; +import org.apache.hudi.common.util.queue.WaitStrategyFactory; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.testutils.HoodieClientTestHarness; @@ -101,7 +102,7 @@ protected Integer getResult() { try { exec = new DisruptorExecutor(hoodieWriteConfig.getWriteBufferLimitBytes(), hoodieRecords.iterator(), consumer, - getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA), getPreExecuteRunnable()); + getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA), WaitStrategyFactory.DEFAULT_STRATEGY, getPreExecuteRunnable()); int result = exec.execute(); // It should buffer and write 100 records assertEquals(128, result); @@ -149,7 +150,7 @@ protected Integer getResult() { AtomicReference actualException = new AtomicReference<>(); try { executor = new DisruptorExecutor(hoodieWriteConfig.getWriteBufferLimitBytes(), hoodieRecords.iterator(), consumer, - getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA), getPreExecuteRunnable()); + getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA), WaitStrategyFactory.DEFAULT_STRATEGY, getPreExecuteRunnable()); DisruptorExecutor>, Integer> finalExecutor = executor; Future future = pool.submit(() -> { diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorMessageQueue.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorMessageQueue.java index 487b67c966468..79893f0be0c63 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorMessageQueue.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorMessageQueue.java @@ -10,6 +10,7 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.queue.BoundedInMemoryQueueConsumer; import org.apache.hudi.common.util.queue.DisruptorExecutor; +import org.apache.hudi.common.util.queue.WaitStrategyFactory; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.testutils.HoodieClientTestHarness; import org.apache.spark.TaskContext; @@ -109,7 +110,7 @@ protected Integer getResult() { try { exec = new DisruptorExecutor(hoodieWriteConfig.getWriteBufferLimitBytes(), hoodieRecords.iterator(), consumer, - getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA), getPreExecuteRunnable()); + getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA), WaitStrategyFactory.DEFAULT_STRATEGY, getPreExecuteRunnable()); int result = exec.execute(); // It should buffer and write 100 records assertEquals(100, result); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorExecutor.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorExecutor.java index cb22b4c8461a4..6c8f6affc51a3 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorExecutor.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorExecutor.java @@ -58,24 +58,24 @@ public class DisruptorExecutor extends HoodieExecutor { // pre-execute function to implement environment specific behavior before executors (producers/consumer) run private final Runnable preExecuteRunnable; - public DisruptorExecutor(final long bufferLimitInBytes, final Iterator inputItr, - BoundedInMemoryQueueConsumer consumer, Function transformFunction, Runnable preExecuteRunnable) { - this(bufferLimitInBytes, new IteratorBasedDisruptorProducer<>(inputItr), Option.of(consumer), transformFunction, 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 long bufferLimitInBytes, DisruptorBasedProducer producer, + public DisruptorExecutor(final int bufferSize, DisruptorBasedProducer producer, Option> consumer, final Function transformFunction) { - this(bufferLimitInBytes, producer, consumer, transformFunction, Functions.noop()); + this(bufferSize, producer, consumer, transformFunction, WaitStrategyFactory.DEFAULT_STRATEGY, Functions.noop()); } - public DisruptorExecutor(final long bufferLimitInBytes, DisruptorBasedProducer producer, - Option> consumer, final Function transformFunction, Runnable preExecuteRunnable) { - this(bufferLimitInBytes, Collections.singletonList(producer), consumer, transformFunction, new DefaultSizeEstimator<>(), preExecuteRunnable); + 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 long bufferLimitInBytes, List> producers, + public DisruptorExecutor(final int bufferSize, List> producers, Option> consumer, final Function transformFunction, - final SizeEstimator sizeEstimator, Runnable preExecuteRunnable) { + final String waitStrategy, Runnable preExecuteRunnable) { this.producers = producers; this.consumer = consumer; this.preExecuteRunnable = preExecuteRunnable; @@ -83,7 +83,7 @@ public DisruptorExecutor(final long bufferLimitInBytes, List(bufferLimitInBytes, transformFunction, sizeEstimator, producers.size()); + this.queue = new DisruptorMessageQueue<>(bufferSize, transformFunction, waitStrategy, producers.size()); } @@ -178,7 +178,7 @@ public boolean isRemaining() { public void shutdownNow() { producerExecutorService.shutdownNow(); consumerExecutorService.shutdownNow(); - queue.shutdownNow(); + queue.close(); } @Override diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageQueue.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageQueue.java index 667cc09a52a34..68f437db6463a 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageQueue.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageQueue.java @@ -1,30 +1,25 @@ package org.apache.hudi.common.util.queue; -import com.lmax.disruptor.BusySpinWaitStrategy; import com.lmax.disruptor.EventTranslator; +import com.lmax.disruptor.WaitStrategy; import com.lmax.disruptor.dsl.Disruptor; import com.lmax.disruptor.dsl.ProducerType; -import org.apache.hudi.common.util.SizeEstimator; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; import java.util.function.Function; public class DisruptorMessageQueue extends HoodieMessageQueue { private final Disruptor> queue; private final Function transformFunction; - private ExecutorService executorService; - private final int bufferSize = 128 * 1024; + public DisruptorMessageQueue(int bufferSize, Function transformFunction, String waitStrategyName, int producerNumber) { - public DisruptorMessageQueue(long bufferLimitInBytes, Function transformFunction, SizeEstimator sizeEstimator, int producerNumber) { - this.executorService = Executors.newCachedThreadPool(); + WaitStrategy waitStrategy = WaitStrategyFactory.build(waitStrategyName); if (producerNumber > 1) { - this.queue = new Disruptor<>(HoodieDisruptorEvent::new, bufferSize, executorService, ProducerType.MULTI, new BusySpinWaitStrategy()); + this.queue = new Disruptor<>(HoodieDisruptorEvent::new, bufferSize, HoodieDaemonThreadFactory.INSTANCE, ProducerType.MULTI, waitStrategy); } else { - this.queue = new Disruptor<>(HoodieDisruptorEvent::new, bufferSize, executorService, ProducerType.SINGLE, new BusySpinWaitStrategy()); + this.queue = new Disruptor<>(HoodieDisruptorEvent::new, bufferSize, HoodieDaemonThreadFactory.INSTANCE, ProducerType.SINGLE, waitStrategy); } this.transformFunction = transformFunction; @@ -59,10 +54,6 @@ public void close() { queue.shutdown(); } - public void shutdownNow() { - executorService.shutdownNow(); - } - public Disruptor> getInnerQueue() { return this.queue; } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/ExecutorType.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/ExecutorType.java new file mode 100644 index 0000000000000..959f9979e0940 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/ExecutorType.java @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.common.util.queue; + +import org.apache.hudi.keygen.constant.KeyGeneratorType; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +/** + * Types of {@link org.apache.hudi.common.util.queue.HoodieExecutor}. + */ +public enum ExecutorType { + + /** + * Executor which orchestrates concurrent producers and consumers communicating through a bounded in-memory message queue using LinkedBlockingQueue. + */ + BOUNDED_IN_MEMORY_EXECUTOR, + + /** + * Executor which orchestrates concurrent producers and consumers communicating through disruptor as a lock free message queue + * to gain better writing performance. Although DisruptorExecutor is still an experimental feature. + */ + DISRUPTOR_EXECUTOR; + + public static List getNames() { + List names = new ArrayList<>(KeyGeneratorType.values().length); + Arrays.stream(KeyGeneratorType.values()) + .forEach(x -> names.add(x.name())); + return names; + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieDaemonThreadFactory.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieDaemonThreadFactory.java new file mode 100644 index 0000000000000..c51da0d9f19fd --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieDaemonThreadFactory.java @@ -0,0 +1,35 @@ +/* + * 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 enum HoodieDaemonThreadFactory implements ThreadFactory { + + INSTANCE; + + @Override + public Thread newThread(@NotNull final Runnable r) { + System.out.println("Creating a new Thread"); + Thread t = new Thread(r); + t.setDaemon(true); + return t; + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieDisruptorEventFactory.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieDisruptorEventFactory.java index 76c8a4c2e5a15..73ae469bc69df 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieDisruptorEventFactory.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieDisruptorEventFactory.java @@ -20,10 +20,10 @@ import com.lmax.disruptor.EventFactory; -public class HoodieDisruptorEventFactory implements EventFactory { +public class HoodieDisruptorEventFactory implements EventFactory> { @Override - public HoodieDisruptorEvent newInstance() { - return new HoodieDisruptorEvent(); + public HoodieDisruptorEvent newInstance() { + return new HoodieDisruptorEvent<>(); } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/WaitStrategyFactory.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/WaitStrategyFactory.java new file mode 100644 index 0000000000000..9492e8b1a88e9 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/WaitStrategyFactory.java @@ -0,0 +1,50 @@ +/* + * 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.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"; + + /** + * Build WaitStrategy for disruptor + */ + public static WaitStrategy build(String name) { + WaitStrategy waitStrategy = null; + if ("BlockingWaitStrategy".equals(name)) { + waitStrategy = new BlockingWaitStrategy(); + } else if ("BusySpinWaitStrategy".equals(name)) { + waitStrategy = new BusySpinWaitStrategy(); + } else if ("SleepingWaitStrategy".equals(name)) { + waitStrategy = new SleepingWaitStrategy(); + } else if ("YieldingWaitStrategy".equals(name)) { + waitStrategy = new YieldingWaitStrategy(); + } else { + throw new HoodieException("Invalid wait strategy: " + name); + } + return waitStrategy; + } +} From 1db793686e949d3fa2f546f52ccde8201797d2b2 Mon Sep 17 00:00:00 2001 From: yuezhang Date: Sat, 23 Apr 2022 15:17:18 +0800 Subject: [PATCH 04/50] need more consuming tuning --- .../TestDisruptorExecutionInSpark.java | 1 - .../common/util/queue/DisruptorExecutor.java | 9 ++++----- .../util/queue/DisruptorMessageHandler.java | 5 +---- .../util/queue/DisruptorMessageQueue.java | 7 ++++--- .../util/queue/HoodieDaemonThreadFactory.java | 17 +++++++++++++---- 5 files changed, 22 insertions(+), 17 deletions(-) diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorExecutionInSpark.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorExecutionInSpark.java index 529fdffe1498c..0c92c5ec4bca6 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorExecutionInSpark.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorExecutionInSpark.java @@ -161,7 +161,6 @@ protected Integer getResult() { } }); - Thread.sleep(2 * 1000); future.cancel(true); future.get(); assertTrue(actualException.get() instanceof HoodieException); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorExecutor.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorExecutor.java index 6c8f6affc51a3..1958ab3ef7c7c 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorExecutor.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorExecutor.java @@ -20,10 +20,8 @@ import com.lmax.disruptor.dsl.Disruptor; import org.apache.hudi.common.util.CustomizedThreadFactory; -import org.apache.hudi.common.util.DefaultSizeEstimator; import org.apache.hudi.common.util.Functions; import org.apache.hudi.common.util.Option; -import org.apache.hudi.common.util.SizeEstimator; import org.apache.hudi.exception.HoodieException; import org.apache.log4j.LogManager; @@ -83,10 +81,9 @@ public DisruptorExecutor(final int bufferSize, List> p 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()); + this.queue = new DisruptorMessageQueue<>(bufferSize, transformFunction, waitStrategy, producers.size(), preExecuteRunnable); } - /** * Start all Producers. */ @@ -117,6 +114,7 @@ public E execute() { Future future = startConsumer(queue.getInnerQueue()); startProducers(); // Wait for consumer to be done + // TODO need to improve return future.get(); } catch (InterruptedException ie) { shutdownNow(); @@ -137,13 +135,14 @@ private Future startConsumer(Disruptor> disruptor) { LOG.info("starting consumer thread"); preExecuteRunnable.run(); try { - DisruptorMessageHandler handler = new DisruptorMessageHandler<>(consumer, preExecuteRunnable); + DisruptorMessageHandler handler = new DisruptorMessageHandler<>(consumer); disruptor.handleEventsWith(handler); // start disruptor queue.getInnerQueue().start(); isRegister.set(true); while (!handler.isFinished()) { + // TODO need to improve Thread.sleep(1 * 1000); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageHandler.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageHandler.java index cbaf70b50b705..cf7fcfeec1b16 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageHandler.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageHandler.java @@ -24,16 +24,13 @@ public class DisruptorMessageHandler implements EventHandler consumer; private boolean finished = false; - private Runnable preExecuteRunnable; - public DisruptorMessageHandler(BoundedInMemoryQueueConsumer consumer, Runnable preExecuteRunnable) { + public DisruptorMessageHandler(BoundedInMemoryQueueConsumer consumer) { this.consumer = consumer; - this.preExecuteRunnable = preExecuteRunnable; } @Override public void onEvent(HoodieDisruptorEvent event, long sequence, boolean endOfBatch) { - preExecuteRunnable.run(); if (event == null || event.get() == null) { // end of ingestion finished = true; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageQueue.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageQueue.java index 68f437db6463a..ddeaed0941db8 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageQueue.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageQueue.java @@ -12,14 +12,15 @@ public class DisruptorMessageQueue extends HoodieMessageQueue { private final Disruptor> queue; private final Function transformFunction; - public DisruptorMessageQueue(int bufferSize, Function transformFunction, String waitStrategyName, int producerNumber) { + 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, HoodieDaemonThreadFactory.INSTANCE, ProducerType.MULTI, waitStrategy); + this.queue = new Disruptor<>(HoodieDisruptorEvent::new, bufferSize, threadFactory, ProducerType.MULTI, waitStrategy); } else { - this.queue = new Disruptor<>(HoodieDisruptorEvent::new, bufferSize, HoodieDaemonThreadFactory.INSTANCE, ProducerType.SINGLE, waitStrategy); + this.queue = new Disruptor<>(HoodieDisruptorEvent::new, bufferSize, threadFactory, ProducerType.SINGLE, waitStrategy); } this.transformFunction = transformFunction; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieDaemonThreadFactory.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieDaemonThreadFactory.java index c51da0d9f19fd..3e3daa7513555 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieDaemonThreadFactory.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieDaemonThreadFactory.java @@ -21,14 +21,23 @@ import org.jetbrains.annotations.NotNull; import java.util.concurrent.ThreadFactory; -public enum HoodieDaemonThreadFactory implements ThreadFactory { +public class HoodieDaemonThreadFactory implements ThreadFactory { - INSTANCE; + private Runnable preExecuteRunnable; + public HoodieDaemonThreadFactory(Runnable preExecuteRunnable) { + this.preExecuteRunnable = preExecuteRunnable; + } @Override public Thread newThread(@NotNull final Runnable r) { - System.out.println("Creating a new Thread"); - Thread t = new Thread(r); + Thread t = new Thread(new Runnable() { + + @Override + public void run() { + preExecuteRunnable.run(); + r.run(); + } + }); t.setDaemon(true); return t; } From 725f5c323b9ce0317aa929de261433b8c72a466e Mon Sep 17 00:00:00 2001 From: yuezhang Date: Sat, 23 Apr 2022 21:58:08 +0800 Subject: [PATCH 05/50] improved && add benchmark --- .../TestDisruptorExecutionInSpark.java | 8 +- .../execution/TestDisruptorMessageQueue.java | 4 +- .../common/util/queue/DisruptorExecutor.java | 68 ++++------ .../util/queue/DisruptorMessageHandler.java | 14 +- .../util/queue/DisruptorMessageQueue.java | 11 +- .../queue/FunctionBasedDisruptorProducer.java | 2 - .../queue/IteratorBasedDisruptorProducer.java | 2 - .../BoundInMemoryExecutorBenchmark.scala | 123 +++++++++++++++++ .../DisruptorExecutorBenchmark.scala | 124 ++++++++++++++++++ 9 files changed, 287 insertions(+), 69 deletions(-) create mode 100644 hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/execution/benchmark/BoundInMemoryExecutorBenchmark.scala create mode 100644 hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/execution/benchmark/DisruptorExecutorBenchmark.scala diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorExecutionInSpark.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorExecutionInSpark.java index 0c92c5ec4bca6..8e63206fcf7f1 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorExecutionInSpark.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorExecutionInSpark.java @@ -78,7 +78,7 @@ public void testExecutor() { final List hoodieRecords = dataGen.generateInserts(instantTime, 128); HoodieWriteConfig hoodieWriteConfig = mock(HoodieWriteConfig.class); - when(hoodieWriteConfig.getWriteBufferLimitBytes()).thenReturn(1024); + when(hoodieWriteConfig.getWriteBufferSize()).thenReturn(8); BoundedInMemoryQueueConsumer, Integer> consumer = new BoundedInMemoryQueueConsumer, Integer>() { @@ -101,7 +101,7 @@ protected Integer getResult() { DisruptorExecutor>, Integer> exec = null; try { - exec = new DisruptorExecutor(hoodieWriteConfig.getWriteBufferLimitBytes(), hoodieRecords.iterator(), consumer, + exec = new DisruptorExecutor(hoodieWriteConfig.getWriteBufferSize(), hoodieRecords.iterator(), consumer, getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA), WaitStrategyFactory.DEFAULT_STRATEGY, getPreExecuteRunnable()); int result = exec.execute(); // It should buffer and write 100 records @@ -121,7 +121,7 @@ public void testInterruptExecutor() { ExecutorService pool = Executors.newSingleThreadExecutor(); HoodieWriteConfig hoodieWriteConfig = mock(HoodieWriteConfig.class); - when(hoodieWriteConfig.getWriteBufferLimitBytes()).thenReturn(1024); + when(hoodieWriteConfig.getWriteBufferSize()).thenReturn(1024); BoundedInMemoryQueueConsumer, Integer> consumer = new BoundedInMemoryQueueConsumer, Integer>() { @@ -149,7 +149,7 @@ protected Integer getResult() { DisruptorExecutor>, Integer> executor = null; AtomicReference actualException = new AtomicReference<>(); try { - executor = new DisruptorExecutor(hoodieWriteConfig.getWriteBufferLimitBytes(), hoodieRecords.iterator(), consumer, + executor = new DisruptorExecutor(hoodieWriteConfig.getWriteBufferSize(), hoodieRecords.iterator(), consumer, getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA), WaitStrategyFactory.DEFAULT_STRATEGY, getPreExecuteRunnable()); DisruptorExecutor>, Integer> finalExecutor = executor; diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorMessageQueue.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorMessageQueue.java index 79893f0be0c63..f7772de4b29d1 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorMessageQueue.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorMessageQueue.java @@ -77,7 +77,7 @@ public void testRecordReading() throws Exception { }); HoodieWriteConfig hoodieWriteConfig = mock(HoodieWriteConfig.class); - when(hoodieWriteConfig.getWriteBufferLimitBytes()).thenReturn(1024); + when(hoodieWriteConfig.getWriteBufferSize()).thenReturn(16); BoundedInMemoryQueueConsumer, Integer> consumer = new BoundedInMemoryQueueConsumer, Integer>() { @@ -109,7 +109,7 @@ protected Integer getResult() { DisruptorExecutor>, Integer> exec = null; try { - exec = new DisruptorExecutor(hoodieWriteConfig.getWriteBufferLimitBytes(), hoodieRecords.iterator(), consumer, + exec = new DisruptorExecutor(hoodieWriteConfig.getWriteBufferSize(), hoodieRecords.iterator(), consumer, getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA), WaitStrategyFactory.DEFAULT_STRATEGY, getPreExecuteRunnable()); int result = exec.execute(); // It should buffer and write 100 records diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorExecutor.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorExecutor.java index 1958ab3ef7c7c..bb23a42e604ba 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorExecutor.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorExecutor.java @@ -30,12 +30,10 @@ import java.util.Collections; import java.util.Iterator; import java.util.List; -import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorCompletionService; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; -import java.util.concurrent.Future; -import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.Function; import java.util.stream.Collectors; @@ -111,11 +109,15 @@ public ExecutorCompletionService startProducers() { @Override public E execute() { try { - Future future = startConsumer(queue.getInnerQueue()); - startProducers(); - // Wait for consumer to be done - // TODO need to improve - return future.get(); + assert consumer.isPresent(); + setupConsumer(); + ExecutorCompletionService pool = startProducers(); + + waitForProducersFinished(pool); + queue.getInnerQueue().shutdown(); + consumer.get().finish(); + + return consumer.get().getResult(); } catch (InterruptedException ie) { shutdownNow(); Thread.currentThread().interrupt(); @@ -125,52 +127,26 @@ public E execute() { } } + private void waitForProducersFinished(ExecutorCompletionService pool) throws InterruptedException, ExecutionException { + for (int i = 0; i < producers.size(); i++ ) { + pool.take().get(); + } + } + /** * Start only consumer. */ - private Future startConsumer(Disruptor> disruptor) { - AtomicBoolean isRegister = new AtomicBoolean(false); - Future future = consumer.map(consumer -> { - return consumerExecutorService.submit(() -> { - LOG.info("starting consumer thread"); - preExecuteRunnable.run(); - try { - DisruptorMessageHandler handler = new DisruptorMessageHandler<>(consumer); - disruptor.handleEventsWith(handler); - - // start disruptor - queue.getInnerQueue().start(); - isRegister.set(true); - while (!handler.isFinished()) { - // TODO need to improve - Thread.sleep(1 * 1000); - } - - LOG.info("Queue Consumption is done; notifying producer threads"); - consumer.finish(); - return consumer.getResult(); - } catch (Exception e) { - LOG.error("error consuming records", e); - throw e; - } - }); - }).orElse(CompletableFuture.completedFuture(null)); - - // waiting until consumer registered. - while (!isRegister.get()) { - try { - Thread.sleep(1 * 1000); - } catch (InterruptedException e) { - // ignore here - } - } + private void setupConsumer() { + DisruptorMessageHandler handler = new DisruptorMessageHandler<>(consumer.get()); - return future; + Disruptor> innerQueue = queue.getInnerQueue(); + innerQueue.handleEventsWith(handler); + innerQueue.start(); } @Override public boolean isRemaining() { - return false; + return !queue.isEmpty(); } @Override diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageHandler.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageHandler.java index cf7fcfeec1b16..47308cba86007 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageHandler.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageHandler.java @@ -23,7 +23,6 @@ public class DisruptorMessageHandler implements EventHandler> { private BoundedInMemoryQueueConsumer consumer; - private boolean finished = false; public DisruptorMessageHandler(BoundedInMemoryQueueConsumer consumer) { this.consumer = consumer; @@ -31,16 +30,7 @@ public DisruptorMessageHandler(BoundedInMemoryQueueConsumer consumer) { @Override public void onEvent(HoodieDisruptorEvent event, long sequence, boolean endOfBatch) { - if (event == null || event.get() == null) { - // end of ingestion - finished = true; - } else { - consumer.consumeOneRecord(event.get()); - event.clear(); - } - } - - public boolean isFinished() { - return finished; + consumer.consumeOneRecord(event.get()); + event.clear(); } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageQueue.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageQueue.java index ddeaed0941db8..b6d9cdec5f1db 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageQueue.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageQueue.java @@ -1,6 +1,7 @@ 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; @@ -9,11 +10,14 @@ public class DisruptorMessageQueue extends HoodieMessageQueue { + /** Interval used for waiting. **/ + public static final int WAIT_INTERVAL_SEC = 1; + 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); @@ -23,6 +27,7 @@ public DisruptorMessageQueue(int bufferSize, Function transformFunction, S this.queue = new Disruptor<>(HoodieDisruptorEvent::new, bufferSize, threadFactory, ProducerType.SINGLE, waitStrategy); } + this.ringBuffer = queue.getRingBuffer(); this.transformFunction = transformFunction; } @@ -58,4 +63,8 @@ public void close() { public Disruptor> getInnerQueue() { return this.queue; } + + public boolean isEmpty() { + return ringBuffer.getBufferSize() == ringBuffer.remainingCapacity(); + } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/FunctionBasedDisruptorProducer.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/FunctionBasedDisruptorProducer.java index 8aa8c6da8eeee..86538687e7c8d 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/FunctionBasedDisruptorProducer.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/FunctionBasedDisruptorProducer.java @@ -38,7 +38,5 @@ public void produce(DisruptorMessageQueue queue) throws Exception { LOG.info("starting function which will enqueue records"); producerFunction.apply(queue); LOG.info("finished function which will enqueue records"); - // poison after buffer finished. - queue.insertRecord(null); } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/IteratorBasedDisruptorProducer.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/IteratorBasedDisruptorProducer.java index 1f1c68ea9c0c5..23bc7705d3f03 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/IteratorBasedDisruptorProducer.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/IteratorBasedDisruptorProducer.java @@ -40,8 +40,6 @@ public void produce(DisruptorMessageQueue queue) throws Exception { while (inputIterator.hasNext()) { queue.insertRecord(inputIterator.next()); } - // poison after buffer finished. - queue.insertRecord(null); LOG.info("finished buffering records"); } } diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/execution/benchmark/BoundInMemoryExecutorBenchmark.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/execution/benchmark/BoundInMemoryExecutorBenchmark.scala new file mode 100644 index 0000000000000..3ab66cc2ac5b2 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/execution/benchmark/BoundInMemoryExecutorBenchmark.scala @@ -0,0 +1,123 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.benchmark + +import java.sql.{Date, Timestamp} + +import org.apache.hadoop.fs.Path +import org.apache.hudi.HoodieSparkUtils +import org.apache.spark.SparkConf +import org.apache.spark.hudi.benchmark.{HoodieBenchmark, HoodieBenchmarkBase} +import org.apache.spark.sql.hudi.HoodieSparkSessionExtension +import org.apache.spark.sql.types._ +import org.apache.spark.sql.{DataFrame, RowFactory, SparkSession} + +import scala.util.Random + +object BoundInMemoryExecutorBenchmark extends HoodieBenchmarkBase { + + protected val spark: SparkSession = getSparkSession + + def getSparkSession: SparkSession = SparkSession.builder() + .master("local[4]") + .appName(this.getClass.getCanonicalName) + .withExtensions(new HoodieSparkSessionExtension) + .config("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + .config("hoodie.insert.shuffle.parallelism", "2") + .config("hoodie.datasource.write.operation", "insert") + .config("hoodie.upsert.shuffle.parallelism", "2") + .config("hoodie.delete.shuffle.parallelism", "2") + .config("spark.sql.session.timeZone", "CTT") + .config(sparkConf()) + .getOrCreate() + + def sparkConf(): SparkConf = { + val sparkConf = new SparkConf() + if (HoodieSparkUtils.gteqSpark3_2) { + sparkConf.set("spark.sql.catalog.spark_catalog", + "org.apache.spark.sql.hudi.catalog.HoodieCatalog") + } + sparkConf + } + + def createHoodieCowTable(tableName: String, tablePath: String) = { + createDataFrame(10000000).registerTempTable("ds") + spark.sql( + s""" + |create table $tableName using hudi + |tblproperties(primaryKey = 'c1') + |location '${tablePath}' + |As + |select * from ds + """.stripMargin) + } + + private def createDataFrame(number: Int): DataFrame = { + val schema = new StructType() + .add("c1", IntegerType) + .add("c11", IntegerType) + .add("c12", IntegerType) + .add("c2", StringType) + .add("c3", DecimalType(38, 10)) + .add("c4", TimestampType) + .add("c5", ShortType) + .add("c6", DateType) + .add("c7", BinaryType) + .add("c9", ByteType) + + val rdd = spark.sparkContext.parallelize(0 to number, 2).map { item => + val c1 = Integer.valueOf(item) + val c11 = Integer.valueOf(Random.nextInt(10000)) + val c12 = Integer.valueOf(Random.nextInt(10000)) + val c2 = s" ${item}abc" + val c3 = new java.math.BigDecimal(s"${Random.nextInt(1000)}.${Random.nextInt(100)}") + val c4 = new Timestamp(System.currentTimeMillis()) + val c5 = java.lang.Short.valueOf(s"${16}") + val c6 = Date.valueOf(s"${2020}-${item % 11 + 1}-${item % 28 + 1}") + val c7 = Array(item).map(_.toByte) + val c8 = java.lang.Byte.valueOf("9") + RowFactory.create(c1, c11, c12, c2, c3, c4, c5, c6, c7, c8) + } + spark.createDataFrame(rdd, schema) + } + + def withTempTable(tableNames: String*)(f: => Unit): Unit = { + try f finally tableNames.foreach(spark.catalog.dropTempView) + } + + private def cowTableDisruptorExecutorBenchmark(tableName: String = "executorBenchmark"): Unit = { + withTempDir {f => + withTempTable(tableName) { + val benchmark = new HoodieBenchmark("COW Ingestion", 10000000) + benchmark.addCase("BoundInMemory Executor") { _ => + createHoodieCowTable(tableName, new Path(f.getCanonicalPath, tableName).toUri.toString) + } + benchmark.run() + } + } + } + + override def afterAll(): Unit = { + spark.stop() + } + + override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { + cowTableDisruptorExecutorBenchmark() + } +} diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/execution/benchmark/DisruptorExecutorBenchmark.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/execution/benchmark/DisruptorExecutorBenchmark.scala new file mode 100644 index 0000000000000..eb91ebac41e72 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/execution/benchmark/DisruptorExecutorBenchmark.scala @@ -0,0 +1,124 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.benchmark + +import java.sql.{Date, Timestamp} + +import org.apache.hadoop.fs.Path +import org.apache.hudi.HoodieSparkUtils +import org.apache.spark.SparkConf +import org.apache.spark.hudi.benchmark.{HoodieBenchmark, HoodieBenchmarkBase} +import org.apache.spark.sql.hudi.HoodieSparkSessionExtension +import org.apache.spark.sql.types._ +import org.apache.spark.sql.{DataFrame, RowFactory, SparkSession} + +import scala.util.Random + +object DisruptorExecutorBenchmark extends HoodieBenchmarkBase { + + protected val spark: SparkSession = getSparkSession + + def getSparkSession: SparkSession = SparkSession.builder() + .master("local[4]") + .appName(this.getClass.getCanonicalName) + .withExtensions(new HoodieSparkSessionExtension) + .config("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + .config("hoodie.insert.shuffle.parallelism", "2") + .config("hoodie.datasource.write.operation", "insert") + .config("hoodie.upsert.shuffle.parallelism", "2") + .config("hoodie.delete.shuffle.parallelism", "2") + .config("spark.sql.session.timeZone", "CTT") + .config("hoodie.write.executor.type", "DISRUPTOR_EXECUTOR") + .config(sparkConf()) + .getOrCreate() + + def sparkConf(): SparkConf = { + val sparkConf = new SparkConf() + if (HoodieSparkUtils.gteqSpark3_2) { + sparkConf.set("spark.sql.catalog.spark_catalog", + "org.apache.spark.sql.hudi.catalog.HoodieCatalog") + } + sparkConf + } + + def createHoodieCowTable(tableName: String, tablePath: String) = { + createDataFrame(10000000).registerTempTable("ds") + spark.sql( + s""" + |create table $tableName using hudi + |tblproperties(primaryKey = 'c1') + |location '${tablePath}' + |As + |select * from ds + """.stripMargin) + } + + private def createDataFrame(number: Int): DataFrame = { + val schema = new StructType() + .add("c1", IntegerType) + .add("c11", IntegerType) + .add("c12", IntegerType) + .add("c2", StringType) + .add("c3", DecimalType(38, 10)) + .add("c4", TimestampType) + .add("c5", ShortType) + .add("c6", DateType) + .add("c7", BinaryType) + .add("c9", ByteType) + + val rdd = spark.sparkContext.parallelize(0 to number, 2).map { item => + val c1 = Integer.valueOf(item) + val c11 = Integer.valueOf(Random.nextInt(10000)) + val c12 = Integer.valueOf(Random.nextInt(10000)) + val c2 = s" ${item}abc" + val c3 = new java.math.BigDecimal(s"${Random.nextInt(1000)}.${Random.nextInt(100)}") + val c4 = new Timestamp(System.currentTimeMillis()) + val c5 = java.lang.Short.valueOf(s"${16}") + val c6 = Date.valueOf(s"${2020}-${item % 11 + 1}-${item % 28 + 1}") + val c7 = Array(item).map(_.toByte) + val c8 = java.lang.Byte.valueOf("9") + RowFactory.create(c1, c11, c12, c2, c3, c4, c5, c6, c7, c8) + } + spark.createDataFrame(rdd, schema) + } + + def withTempTable(tableNames: String*)(f: => Unit): Unit = { + try f finally tableNames.foreach(spark.catalog.dropTempView) + } + + private def cowTableDisruptorExecutorBenchmark(tableName: String = "executorBenchmark"): Unit = { + withTempDir {f => + withTempTable(tableName) { + val benchmark = new HoodieBenchmark("COW Ingestion", 10000000) + benchmark.addCase("Disruptor Executor") { _ => + createHoodieCowTable(tableName, new Path(f.getCanonicalPath, tableName).toUri.toString) + } + benchmark.run() + } + } + } + + override def afterAll(): Unit = { + spark.stop() + } + + override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { + cowTableDisruptorExecutorBenchmark() + } +} From 71a9f13768b6fd6224809223d1b63c9ec96e8362 Mon Sep 17 00:00:00 2001 From: yuezhang Date: Sat, 23 Apr 2022 22:59:24 +0800 Subject: [PATCH 06/50] adjust bulk_insert in benchmark --- .../benchmark/BoundInMemoryExecutorBenchmark.scala | 4 +++- .../execution/benchmark/DisruptorExecutorBenchmark.scala | 6 ++++-- 2 files changed, 7 insertions(+), 3 deletions(-) diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/execution/benchmark/BoundInMemoryExecutorBenchmark.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/execution/benchmark/BoundInMemoryExecutorBenchmark.scala index 3ab66cc2ac5b2..ee015f63c0fac 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/execution/benchmark/BoundInMemoryExecutorBenchmark.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/execution/benchmark/BoundInMemoryExecutorBenchmark.scala @@ -40,7 +40,9 @@ object BoundInMemoryExecutorBenchmark extends HoodieBenchmarkBase { .withExtensions(new HoodieSparkSessionExtension) .config("spark.serializer", "org.apache.spark.serializer.KryoSerializer") .config("hoodie.insert.shuffle.parallelism", "2") - .config("hoodie.datasource.write.operation", "insert") + .config("hoodie.datasource.write.operation", "bulk_insert") + .config("hoodie.datasource.write.row.writer.enable", "false") + .config("hoodie.bulkinsert.shuffle.parallelism", "10") .config("hoodie.upsert.shuffle.parallelism", "2") .config("hoodie.delete.shuffle.parallelism", "2") .config("spark.sql.session.timeZone", "CTT") diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/execution/benchmark/DisruptorExecutorBenchmark.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/execution/benchmark/DisruptorExecutorBenchmark.scala index eb91ebac41e72..33c28048096c4 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/execution/benchmark/DisruptorExecutorBenchmark.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/execution/benchmark/DisruptorExecutorBenchmark.scala @@ -40,11 +40,13 @@ object DisruptorExecutorBenchmark extends HoodieBenchmarkBase { .withExtensions(new HoodieSparkSessionExtension) .config("spark.serializer", "org.apache.spark.serializer.KryoSerializer") .config("hoodie.insert.shuffle.parallelism", "2") - .config("hoodie.datasource.write.operation", "insert") + .config("hoodie.datasource.write.operation", "bulk_insert") + .config("hoodie.datasource.write.row.writer.enable", "false") + .config("hoodie.bulkinsert.shuffle.parallelism", "10") + .config("hoodie.write.executor.type", "DISRUPTOR_EXECUTOR") .config("hoodie.upsert.shuffle.parallelism", "2") .config("hoodie.delete.shuffle.parallelism", "2") .config("spark.sql.session.timeZone", "CTT") - .config("hoodie.write.executor.type", "DISRUPTOR_EXECUTOR") .config(sparkConf()) .getOrCreate() From b5843f0f84bc5dfdea68fb7bfb8d446905a74d16 Mon Sep 17 00:00:00 2001 From: yuezhang Date: Sat, 23 Apr 2022 23:01:10 +0800 Subject: [PATCH 07/50] adjust bulk_insert in benchmark --- .../execution/benchmark/BoundInMemoryExecutorBenchmark.scala | 2 +- .../sql/execution/benchmark/DisruptorExecutorBenchmark.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/execution/benchmark/BoundInMemoryExecutorBenchmark.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/execution/benchmark/BoundInMemoryExecutorBenchmark.scala index ee015f63c0fac..d3b7089d4600c 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/execution/benchmark/BoundInMemoryExecutorBenchmark.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/execution/benchmark/BoundInMemoryExecutorBenchmark.scala @@ -35,7 +35,7 @@ object BoundInMemoryExecutorBenchmark extends HoodieBenchmarkBase { protected val spark: SparkSession = getSparkSession def getSparkSession: SparkSession = SparkSession.builder() - .master("local[4]") + .master("local[*]") .appName(this.getClass.getCanonicalName) .withExtensions(new HoodieSparkSessionExtension) .config("spark.serializer", "org.apache.spark.serializer.KryoSerializer") diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/execution/benchmark/DisruptorExecutorBenchmark.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/execution/benchmark/DisruptorExecutorBenchmark.scala index 33c28048096c4..650bc92f0cbf7 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/execution/benchmark/DisruptorExecutorBenchmark.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/execution/benchmark/DisruptorExecutorBenchmark.scala @@ -35,7 +35,7 @@ object DisruptorExecutorBenchmark extends HoodieBenchmarkBase { protected val spark: SparkSession = getSparkSession def getSparkSession: SparkSession = SparkSession.builder() - .master("local[4]") + .master("local[*]") .appName(this.getClass.getCanonicalName) .withExtensions(new HoodieSparkSessionExtension) .config("spark.serializer", "org.apache.spark.serializer.KryoSerializer") From 4a1f01ca06a7097e64f34b59770327c807f8aa14 Mon Sep 17 00:00:00 2001 From: yuezhang Date: Sun, 24 Apr 2022 10:15:17 +0800 Subject: [PATCH 08/50] modify benchmark --- .../BoundInMemoryExecutorBenchmark.scala | 63 +++++---- .../DisruptorExecutorBenchmark.scala | 126 ------------------ 2 files changed, 34 insertions(+), 155 deletions(-) delete mode 100644 hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/execution/benchmark/DisruptorExecutorBenchmark.scala diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/execution/benchmark/BoundInMemoryExecutorBenchmark.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/execution/benchmark/BoundInMemoryExecutorBenchmark.scala index d3b7089d4600c..cecd9e4feba4b 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/execution/benchmark/BoundInMemoryExecutorBenchmark.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/execution/benchmark/BoundInMemoryExecutorBenchmark.scala @@ -26,7 +26,7 @@ import org.apache.spark.SparkConf import org.apache.spark.hudi.benchmark.{HoodieBenchmark, HoodieBenchmarkBase} import org.apache.spark.sql.hudi.HoodieSparkSessionExtension import org.apache.spark.sql.types._ -import org.apache.spark.sql.{DataFrame, RowFactory, SparkSession} +import org.apache.spark.sql.{DataFrame, RowFactory, SaveMode, SparkSession} import scala.util.Random @@ -39,12 +39,6 @@ object BoundInMemoryExecutorBenchmark extends HoodieBenchmarkBase { .appName(this.getClass.getCanonicalName) .withExtensions(new HoodieSparkSessionExtension) .config("spark.serializer", "org.apache.spark.serializer.KryoSerializer") - .config("hoodie.insert.shuffle.parallelism", "2") - .config("hoodie.datasource.write.operation", "bulk_insert") - .config("hoodie.datasource.write.row.writer.enable", "false") - .config("hoodie.bulkinsert.shuffle.parallelism", "10") - .config("hoodie.upsert.shuffle.parallelism", "2") - .config("hoodie.delete.shuffle.parallelism", "2") .config("spark.sql.session.timeZone", "CTT") .config(sparkConf()) .getOrCreate() @@ -58,18 +52,6 @@ object BoundInMemoryExecutorBenchmark extends HoodieBenchmarkBase { sparkConf } - def createHoodieCowTable(tableName: String, tablePath: String) = { - createDataFrame(10000000).registerTempTable("ds") - spark.sql( - s""" - |create table $tableName using hudi - |tblproperties(primaryKey = 'c1') - |location '${tablePath}' - |As - |select * from ds - """.stripMargin) - } - private def createDataFrame(number: Int): DataFrame = { val schema = new StructType() .add("c1", IntegerType) @@ -99,19 +81,42 @@ object BoundInMemoryExecutorBenchmark extends HoodieBenchmarkBase { spark.createDataFrame(rdd, schema) } - def withTempTable(tableNames: String*)(f: => Unit): Unit = { - try f finally tableNames.foreach(spark.catalog.dropTempView) - } - private def cowTableDisruptorExecutorBenchmark(tableName: String = "executorBenchmark"): Unit = { withTempDir {f => - withTempTable(tableName) { - val benchmark = new HoodieBenchmark("COW Ingestion", 10000000) - benchmark.addCase("BoundInMemory Executor") { _ => - createHoodieCowTable(tableName, new Path(f.getCanonicalPath, tableName).toUri.toString) - } - benchmark.run() + val benchmark = new HoodieBenchmark("COW Ingestion", 10000000) + benchmark.addCase("BoundInMemory Executor") { _ => + val finalTableName = tableName + Random.nextInt(10000) + val df = createDataFrame(10000000) + df.write.format("hudi") + .mode(SaveMode.Overwrite) + .option("hoodie.datasource.write.recordkey.field", "c1") + .option("hoodie.table.name", finalTableName) + .option("hoodie.insert.shuffle.parallelism", "2") + .option("hoodie.datasource.write.operation", "bulk_insert") + .option("hoodie.datasource.write.row.writer.enable", "false") + .option("hoodie.bulkinsert.shuffle.parallelism", "10") + .option("hoodie.upsert.shuffle.parallelism", "2") + .option("hoodie.delete.shuffle.parallelism", "2") + .save(new Path(f.getCanonicalPath, finalTableName).toUri.toString) + } + + benchmark.addCase("Disruptor Executor") { _ => + val finalTableName = tableName + Random.nextInt(10000) + val df = createDataFrame(10000000) + df.write.format("hudi") + .mode(SaveMode.Overwrite) + .option("hoodie.datasource.write.recordkey.field", "c1") + .option("hoodie.table.name", finalTableName) + .option("hoodie.insert.shuffle.parallelism", "2") + .option("hoodie.datasource.write.operation", "bulk_insert") + .option("hoodie.datasource.write.row.writer.enable", "false") + .option("hoodie.bulkinsert.shuffle.parallelism", "10") + .option("hoodie.upsert.shuffle.parallelism", "2") + .option("hoodie.delete.shuffle.parallelism", "2") + .option("hoodie.write.executor.type", "DISRUPTOR_EXECUTOR") + .save(new Path(f.getCanonicalPath, finalTableName).toUri.toString) } + benchmark.run() } } diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/execution/benchmark/DisruptorExecutorBenchmark.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/execution/benchmark/DisruptorExecutorBenchmark.scala deleted file mode 100644 index 650bc92f0cbf7..0000000000000 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/execution/benchmark/DisruptorExecutorBenchmark.scala +++ /dev/null @@ -1,126 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.execution.benchmark - -import java.sql.{Date, Timestamp} - -import org.apache.hadoop.fs.Path -import org.apache.hudi.HoodieSparkUtils -import org.apache.spark.SparkConf -import org.apache.spark.hudi.benchmark.{HoodieBenchmark, HoodieBenchmarkBase} -import org.apache.spark.sql.hudi.HoodieSparkSessionExtension -import org.apache.spark.sql.types._ -import org.apache.spark.sql.{DataFrame, RowFactory, SparkSession} - -import scala.util.Random - -object DisruptorExecutorBenchmark extends HoodieBenchmarkBase { - - protected val spark: SparkSession = getSparkSession - - def getSparkSession: SparkSession = SparkSession.builder() - .master("local[*]") - .appName(this.getClass.getCanonicalName) - .withExtensions(new HoodieSparkSessionExtension) - .config("spark.serializer", "org.apache.spark.serializer.KryoSerializer") - .config("hoodie.insert.shuffle.parallelism", "2") - .config("hoodie.datasource.write.operation", "bulk_insert") - .config("hoodie.datasource.write.row.writer.enable", "false") - .config("hoodie.bulkinsert.shuffle.parallelism", "10") - .config("hoodie.write.executor.type", "DISRUPTOR_EXECUTOR") - .config("hoodie.upsert.shuffle.parallelism", "2") - .config("hoodie.delete.shuffle.parallelism", "2") - .config("spark.sql.session.timeZone", "CTT") - .config(sparkConf()) - .getOrCreate() - - def sparkConf(): SparkConf = { - val sparkConf = new SparkConf() - if (HoodieSparkUtils.gteqSpark3_2) { - sparkConf.set("spark.sql.catalog.spark_catalog", - "org.apache.spark.sql.hudi.catalog.HoodieCatalog") - } - sparkConf - } - - def createHoodieCowTable(tableName: String, tablePath: String) = { - createDataFrame(10000000).registerTempTable("ds") - spark.sql( - s""" - |create table $tableName using hudi - |tblproperties(primaryKey = 'c1') - |location '${tablePath}' - |As - |select * from ds - """.stripMargin) - } - - private def createDataFrame(number: Int): DataFrame = { - val schema = new StructType() - .add("c1", IntegerType) - .add("c11", IntegerType) - .add("c12", IntegerType) - .add("c2", StringType) - .add("c3", DecimalType(38, 10)) - .add("c4", TimestampType) - .add("c5", ShortType) - .add("c6", DateType) - .add("c7", BinaryType) - .add("c9", ByteType) - - val rdd = spark.sparkContext.parallelize(0 to number, 2).map { item => - val c1 = Integer.valueOf(item) - val c11 = Integer.valueOf(Random.nextInt(10000)) - val c12 = Integer.valueOf(Random.nextInt(10000)) - val c2 = s" ${item}abc" - val c3 = new java.math.BigDecimal(s"${Random.nextInt(1000)}.${Random.nextInt(100)}") - val c4 = new Timestamp(System.currentTimeMillis()) - val c5 = java.lang.Short.valueOf(s"${16}") - val c6 = Date.valueOf(s"${2020}-${item % 11 + 1}-${item % 28 + 1}") - val c7 = Array(item).map(_.toByte) - val c8 = java.lang.Byte.valueOf("9") - RowFactory.create(c1, c11, c12, c2, c3, c4, c5, c6, c7, c8) - } - spark.createDataFrame(rdd, schema) - } - - def withTempTable(tableNames: String*)(f: => Unit): Unit = { - try f finally tableNames.foreach(spark.catalog.dropTempView) - } - - private def cowTableDisruptorExecutorBenchmark(tableName: String = "executorBenchmark"): Unit = { - withTempDir {f => - withTempTable(tableName) { - val benchmark = new HoodieBenchmark("COW Ingestion", 10000000) - benchmark.addCase("Disruptor Executor") { _ => - createHoodieCowTable(tableName, new Path(f.getCanonicalPath, tableName).toUri.toString) - } - benchmark.run() - } - } - } - - override def afterAll(): Unit = { - spark.stop() - } - - override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { - cowTableDisruptorExecutorBenchmark() - } -} From 4b9452370f1da41afb5f17f361989fed8e43f0af Mon Sep 17 00:00:00 2001 From: yuezhang Date: Sun, 24 Apr 2022 11:08:59 +0800 Subject: [PATCH 09/50] adjust --- .../main/java/org/apache/hudi/config/HoodieWriteConfig.java | 2 +- .../benchmark/BoundInMemoryExecutorBenchmark.scala | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java index d0b2787f269b2..c84eee7428d90 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java @@ -243,7 +243,7 @@ public class HoodieWriteConfig extends HoodieConfig { public static final ConfigProperty WRITE_BUFFER_SIZE = ConfigProperty .key("hoodie.write.buffer.size") - .defaultValue(128 * 1024) + .defaultValue(1024) .withDocumentation("The size of the Disruptor Executor ring buffer, must be power of 2"); public static final ConfigProperty WRITE_WAIT_STRATEGY = ConfigProperty diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/execution/benchmark/BoundInMemoryExecutorBenchmark.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/execution/benchmark/BoundInMemoryExecutorBenchmark.scala index cecd9e4feba4b..ac9a604e54e1f 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/execution/benchmark/BoundInMemoryExecutorBenchmark.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/execution/benchmark/BoundInMemoryExecutorBenchmark.scala @@ -83,10 +83,10 @@ object BoundInMemoryExecutorBenchmark extends HoodieBenchmarkBase { private def cowTableDisruptorExecutorBenchmark(tableName: String = "executorBenchmark"): Unit = { withTempDir {f => - val benchmark = new HoodieBenchmark("COW Ingestion", 10000000) + val benchmark = new HoodieBenchmark("COW Ingestion", 5000000) benchmark.addCase("BoundInMemory Executor") { _ => val finalTableName = tableName + Random.nextInt(10000) - val df = createDataFrame(10000000) + val df = createDataFrame(5000000) df.write.format("hudi") .mode(SaveMode.Overwrite) .option("hoodie.datasource.write.recordkey.field", "c1") @@ -102,7 +102,7 @@ object BoundInMemoryExecutorBenchmark extends HoodieBenchmarkBase { benchmark.addCase("Disruptor Executor") { _ => val finalTableName = tableName + Random.nextInt(10000) - val df = createDataFrame(10000000) + val df = createDataFrame(5000000) df.write.format("hudi") .mode(SaveMode.Overwrite) .option("hoodie.datasource.write.recordkey.field", "c1") From 69904cc04bedce8a68ff0683f760cef959db643a Mon Sep 17 00:00:00 2001 From: yuezhang Date: Sun, 24 Apr 2022 11:46:16 +0800 Subject: [PATCH 10/50] tuning --- .../benchmark/BoundInMemoryExecutorBenchmark.scala | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/execution/benchmark/BoundInMemoryExecutorBenchmark.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/execution/benchmark/BoundInMemoryExecutorBenchmark.scala index ac9a604e54e1f..40104c1c7d8dc 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/execution/benchmark/BoundInMemoryExecutorBenchmark.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/execution/benchmark/BoundInMemoryExecutorBenchmark.scala @@ -91,6 +91,9 @@ object BoundInMemoryExecutorBenchmark extends HoodieBenchmarkBase { .mode(SaveMode.Overwrite) .option("hoodie.datasource.write.recordkey.field", "c1") .option("hoodie.table.name", finalTableName) + .option("hoodie.metadata.enable", "false") + .option("hoodie.clean.automatic", "false") + .option("hoodie.bulkinsert.sort.mode", "NONE") .option("hoodie.insert.shuffle.parallelism", "2") .option("hoodie.datasource.write.operation", "bulk_insert") .option("hoodie.datasource.write.row.writer.enable", "false") @@ -107,6 +110,9 @@ object BoundInMemoryExecutorBenchmark extends HoodieBenchmarkBase { .mode(SaveMode.Overwrite) .option("hoodie.datasource.write.recordkey.field", "c1") .option("hoodie.table.name", finalTableName) + .option("hoodie.metadata.enable", "false") + .option("hoodie.clean.automatic", "false") + .option("hoodie.bulkinsert.sort.mode", "NONE") .option("hoodie.insert.shuffle.parallelism", "2") .option("hoodie.datasource.write.operation", "bulk_insert") .option("hoodie.datasource.write.row.writer.enable", "false") From 204abe26ea9cb52e58309866c9e3eec1726eee82 Mon Sep 17 00:00:00 2001 From: yuezhang Date: Sun, 24 Apr 2022 13:06:10 +0800 Subject: [PATCH 11/50] add executor bench mark --- .../testutils/HoodieExecutorTestUtils.java | 61 ++++++++++ .../benchmark/HoodieExecutorBenchmark.scala | 107 ++++++++++++++++++ 2 files changed, 168 insertions(+) create mode 100644 hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/testutils/HoodieExecutorTestUtils.java create mode 100644 hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/hudi/benchmark/HoodieExecutorBenchmark.scala diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/testutils/HoodieExecutorTestUtils.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/testutils/HoodieExecutorTestUtils.java new file mode 100644 index 0000000000000..5528649f945b9 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/testutils/HoodieExecutorTestUtils.java @@ -0,0 +1,61 @@ +/* + * 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.testutils; + +import org.apache.avro.Schema; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.testutils.HoodieTestDataGenerator; +import org.apache.hudi.common.util.CollectionUtils; +import org.apache.hudi.common.util.queue.BoundedInMemoryExecutor; +import org.apache.hudi.common.util.queue.BoundedInMemoryQueueConsumer; +import org.apache.hudi.common.util.queue.DisruptorExecutor; +import org.apache.hudi.common.util.queue.WaitStrategyFactory; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.execution.HoodieLazyInsertIterable; +import org.apache.spark.TaskContext; +import org.apache.spark.TaskContext$; + +import java.util.List; +import java.util.function.Function; + +public class HoodieExecutorTestUtils { + + public DisruptorExecutor getDisruptorExecutor(List hoodieRecords, + BoundedInMemoryQueueConsumer, Integer> consumer) { + return new DisruptorExecutor(HoodieWriteConfig.WRITE_BUFFER_SIZE.defaultValue(), hoodieRecords.iterator(), consumer, + getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA), WaitStrategyFactory.DEFAULT_STRATEGY, getPreExecuteRunnable()); + } + + public BoundedInMemoryExecutor getBoundedInMemoryExecutor(List hoodieRecords, + BoundedInMemoryQueueConsumer, Integer> consumer) { + return new BoundedInMemoryExecutor(Integer.parseInt(HoodieWriteConfig.WRITE_BUFFER_LIMIT_BYTES_VALUE.defaultValue()), hoodieRecords.iterator(), consumer, + getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA), getPreExecuteRunnable()); + } + + static Function, HoodieLazyInsertIterable.HoodieInsertValueGenResult> getTransformFunction( + Schema schema) { + return hoodieRecord -> new HoodieLazyInsertIterable.HoodieInsertValueGenResult(hoodieRecord, schema, CollectionUtils.EMPTY_PROPERTIES); + } + + private Runnable getPreExecuteRunnable() { + final TaskContext taskContext = TaskContext.get(); + return () -> TaskContext$.MODULE$.setTaskContext(taskContext); + } +} diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/hudi/benchmark/HoodieExecutorBenchmark.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/hudi/benchmark/HoodieExecutorBenchmark.scala new file mode 100644 index 0000000000000..5d21bb54791d5 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/hudi/benchmark/HoodieExecutorBenchmark.scala @@ -0,0 +1,107 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.hudi.benchmark + +import org.apache.hudi.common.model.HoodieRecord +import org.apache.hudi.common.table.timeline.HoodieActiveTimeline +import org.apache.hudi.common.testutils.HoodieTestDataGenerator +import org.apache.hudi.common.util.queue.BoundedInMemoryQueueConsumer +import org.apache.hudi.execution.HoodieLazyInsertIterable +import org.apache.hudi.testutils.HoodieExecutorTestUtils + +object HoodieExecutorBenchmark extends HoodieBenchmarkBase { + + val dataGen = new HoodieTestDataGenerator + + val utils = new HoodieExecutorTestUtils + + private def cowTableDisruptorExecutorBenchmark(tableName: String = "executorBenchmark"): Unit = { + val benchmark = new HoodieBenchmark("COW Ingestion", 100000) + benchmark.addCase("Disruptor Executor") { _ => + val con: BoundedInMemoryQueueConsumer[HoodieLazyInsertIterable.HoodieInsertValueGenResult[HoodieRecord[_]], Integer] = new BoundedInMemoryQueueConsumer[HoodieLazyInsertIterable.HoodieInsertValueGenResult[HoodieRecord[_]], Integer]() { + + var count = 0 + /** + * Consumer One record. + */ + override protected def consumeOneRecord(record: HoodieLazyInsertIterable.HoodieInsertValueGenResult[HoodieRecord[_]]): Unit = { + count = count + 1 + } + + /** + * Notifies implementation that we have exhausted consuming records from queue. + */ + override protected def finish(): Unit = { + } + + /** + * Return result of consuming records so far. + */ + override def getResult: Integer = { + return count; + } + } + val instantTime = HoodieActiveTimeline.createNewInstantTime + val hoodieRecords = dataGen.generateInserts(instantTime, 100000) + val disruptorExecutor = utils.getDisruptorExecutor(hoodieRecords, con) + disruptorExecutor.execute() + disruptorExecutor.shutdownNow() + } + + benchmark.addCase("BoundInMemory Executor") { _ => + val con: BoundedInMemoryQueueConsumer[HoodieLazyInsertIterable.HoodieInsertValueGenResult[HoodieRecord[_]], Integer] = new BoundedInMemoryQueueConsumer[HoodieLazyInsertIterable.HoodieInsertValueGenResult[HoodieRecord[_]], Integer]() { + + var count = 0 + /** + * Consumer One record. + */ + override protected def consumeOneRecord(record: HoodieLazyInsertIterable.HoodieInsertValueGenResult[HoodieRecord[_]]): Unit = { + count = count + 1 + } + + /** + * Notifies implementation that we have exhausted consuming records from queue. + */ + override protected def finish(): Unit = { + } + + /** + * Return result of consuming records so far. + */ + override def getResult: Integer = { + count + } + } + val instantTime = HoodieActiveTimeline.createNewInstantTime + val hoodieRecords = dataGen.generateInserts(instantTime, 100000) + val boundInMemoryExecutor = utils.getBoundedInMemoryExecutor(hoodieRecords, con) + boundInMemoryExecutor.execute() + boundInMemoryExecutor.shutdownNow() + } + + benchmark.run() + } + + override def afterAll(): Unit = { + } + + override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { + cowTableDisruptorExecutorBenchmark() + } +} From 45979df221fa815cb744e287dcc806519a9000f8 Mon Sep 17 00:00:00 2001 From: yuezhang Date: Sun, 24 Apr 2022 13:08:47 +0800 Subject: [PATCH 12/50] add recordsNumber to control --- .../spark/hudi/benchmark/HoodieExecutorBenchmark.scala | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/hudi/benchmark/HoodieExecutorBenchmark.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/hudi/benchmark/HoodieExecutorBenchmark.scala index 5d21bb54791d5..5246f3bea1928 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/hudi/benchmark/HoodieExecutorBenchmark.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/hudi/benchmark/HoodieExecutorBenchmark.scala @@ -30,9 +30,10 @@ object HoodieExecutorBenchmark extends HoodieBenchmarkBase { val dataGen = new HoodieTestDataGenerator val utils = new HoodieExecutorTestUtils + val recordsNumber = 10000000 private def cowTableDisruptorExecutorBenchmark(tableName: String = "executorBenchmark"): Unit = { - val benchmark = new HoodieBenchmark("COW Ingestion", 100000) + val benchmark = new HoodieBenchmark("COW Ingestion", recordsNumber) benchmark.addCase("Disruptor Executor") { _ => val con: BoundedInMemoryQueueConsumer[HoodieLazyInsertIterable.HoodieInsertValueGenResult[HoodieRecord[_]], Integer] = new BoundedInMemoryQueueConsumer[HoodieLazyInsertIterable.HoodieInsertValueGenResult[HoodieRecord[_]], Integer]() { @@ -58,7 +59,7 @@ object HoodieExecutorBenchmark extends HoodieBenchmarkBase { } } val instantTime = HoodieActiveTimeline.createNewInstantTime - val hoodieRecords = dataGen.generateInserts(instantTime, 100000) + val hoodieRecords = dataGen.generateInserts(instantTime, recordsNumber) val disruptorExecutor = utils.getDisruptorExecutor(hoodieRecords, con) disruptorExecutor.execute() disruptorExecutor.shutdownNow() @@ -89,7 +90,7 @@ object HoodieExecutorBenchmark extends HoodieBenchmarkBase { } } val instantTime = HoodieActiveTimeline.createNewInstantTime - val hoodieRecords = dataGen.generateInserts(instantTime, 100000) + val hoodieRecords = dataGen.generateInserts(instantTime, recordsNumber) val boundInMemoryExecutor = utils.getBoundedInMemoryExecutor(hoodieRecords, con) boundInMemoryExecutor.execute() boundInMemoryExecutor.shutdownNow() From 075cab1e659ea1967682bfb33ceffe9148fd84c4 Mon Sep 17 00:00:00 2001 From: yuezhang Date: Sun, 24 Apr 2022 13:12:14 +0800 Subject: [PATCH 13/50] code style --- .../apache/spark/hudi/benchmark/HoodieExecutorBenchmark.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/hudi/benchmark/HoodieExecutorBenchmark.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/hudi/benchmark/HoodieExecutorBenchmark.scala index 5246f3bea1928..1a26fad94c787 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/hudi/benchmark/HoodieExecutorBenchmark.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/hudi/benchmark/HoodieExecutorBenchmark.scala @@ -55,7 +55,7 @@ object HoodieExecutorBenchmark extends HoodieBenchmarkBase { * Return result of consuming records so far. */ override def getResult: Integer = { - return count; + count } } val instantTime = HoodieActiveTimeline.createNewInstantTime From e87bfbc81f4a34ebd536b9fe8b20bd6fe1d83493 Mon Sep 17 00:00:00 2001 From: yuezhang Date: Sun, 24 Apr 2022 13:30:31 +0800 Subject: [PATCH 14/50] modify benchmark --- .../benchmark/HoodieExecutorBenchmark.scala | 2 +- .../BoundInMemoryExecutorBenchmark.scala | 43 ++++++++----------- 2 files changed, 19 insertions(+), 26 deletions(-) diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/hudi/benchmark/HoodieExecutorBenchmark.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/hudi/benchmark/HoodieExecutorBenchmark.scala index 1a26fad94c787..afec08e0274c4 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/hudi/benchmark/HoodieExecutorBenchmark.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/hudi/benchmark/HoodieExecutorBenchmark.scala @@ -30,7 +30,7 @@ object HoodieExecutorBenchmark extends HoodieBenchmarkBase { val dataGen = new HoodieTestDataGenerator val utils = new HoodieExecutorTestUtils - val recordsNumber = 10000000 + val recordsNumber = 1000000 private def cowTableDisruptorExecutorBenchmark(tableName: String = "executorBenchmark"): Unit = { val benchmark = new HoodieBenchmark("COW Ingestion", recordsNumber) diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/execution/benchmark/BoundInMemoryExecutorBenchmark.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/execution/benchmark/BoundInMemoryExecutorBenchmark.scala index 40104c1c7d8dc..e5d9c69337683 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/execution/benchmark/BoundInMemoryExecutorBenchmark.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/execution/benchmark/BoundInMemoryExecutorBenchmark.scala @@ -18,8 +18,6 @@ package org.apache.spark.sql.execution.benchmark -import java.sql.{Date, Timestamp} - import org.apache.hadoop.fs.Path import org.apache.hudi.HoodieSparkUtils import org.apache.spark.SparkConf @@ -34,6 +32,9 @@ object BoundInMemoryExecutorBenchmark extends HoodieBenchmarkBase { protected val spark: SparkSession = getSparkSession + // 5000 good local + val recordNumber = 1000000 + def getSparkSession: SparkSession = SparkSession.builder() .master("local[*]") .appName(this.getClass.getCanonicalName) @@ -55,41 +56,26 @@ object BoundInMemoryExecutorBenchmark extends HoodieBenchmarkBase { private def createDataFrame(number: Int): DataFrame = { val schema = new StructType() .add("c1", IntegerType) - .add("c11", IntegerType) - .add("c12", IntegerType) .add("c2", StringType) - .add("c3", DecimalType(38, 10)) - .add("c4", TimestampType) - .add("c5", ShortType) - .add("c6", DateType) - .add("c7", BinaryType) - .add("c9", ByteType) val rdd = spark.sparkContext.parallelize(0 to number, 2).map { item => val c1 = Integer.valueOf(item) - val c11 = Integer.valueOf(Random.nextInt(10000)) - val c12 = Integer.valueOf(Random.nextInt(10000)) - val c2 = s" ${item}abc" - val c3 = new java.math.BigDecimal(s"${Random.nextInt(1000)}.${Random.nextInt(100)}") - val c4 = new Timestamp(System.currentTimeMillis()) - val c5 = java.lang.Short.valueOf(s"${16}") - val c6 = Date.valueOf(s"${2020}-${item % 11 + 1}-${item % 28 + 1}") - val c7 = Array(item).map(_.toByte) - val c8 = java.lang.Byte.valueOf("9") - RowFactory.create(c1, c11, c12, c2, c3, c4, c5, c6, c7, c8) + val c2 = s"abc" + RowFactory.create(c1, c2) } spark.createDataFrame(rdd, schema) } private def cowTableDisruptorExecutorBenchmark(tableName: String = "executorBenchmark"): Unit = { withTempDir {f => - val benchmark = new HoodieBenchmark("COW Ingestion", 5000000) + val benchmark = new HoodieBenchmark("COW Ingestion", recordNumber) benchmark.addCase("BoundInMemory Executor") { _ => val finalTableName = tableName + Random.nextInt(10000) - val df = createDataFrame(5000000) + val df = createDataFrame(recordNumber) df.write.format("hudi") .mode(SaveMode.Overwrite) .option("hoodie.datasource.write.recordkey.field", "c1") + .option("hoodie.datasource.write.partitionpath.field", "c2") .option("hoodie.table.name", finalTableName) .option("hoodie.metadata.enable", "false") .option("hoodie.clean.automatic", "false") @@ -97,18 +83,22 @@ object BoundInMemoryExecutorBenchmark extends HoodieBenchmarkBase { .option("hoodie.insert.shuffle.parallelism", "2") .option("hoodie.datasource.write.operation", "bulk_insert") .option("hoodie.datasource.write.row.writer.enable", "false") - .option("hoodie.bulkinsert.shuffle.parallelism", "10") + .option("hoodie.bulkinsert.shuffle.parallelism", "1") .option("hoodie.upsert.shuffle.parallelism", "2") .option("hoodie.delete.shuffle.parallelism", "2") + .option("hoodie.populate.meta.fields", "false") + .option("hoodie.table.keygenerator.class", "org.apache.hudi.keygen.SimpleKeyGenerator") + .save(new Path(f.getCanonicalPath, finalTableName).toUri.toString) } benchmark.addCase("Disruptor Executor") { _ => val finalTableName = tableName + Random.nextInt(10000) - val df = createDataFrame(5000000) + val df = createDataFrame(recordNumber) df.write.format("hudi") .mode(SaveMode.Overwrite) .option("hoodie.datasource.write.recordkey.field", "c1") + .option("hoodie.datasource.write.partitionpath.field", "c2") .option("hoodie.table.name", finalTableName) .option("hoodie.metadata.enable", "false") .option("hoodie.clean.automatic", "false") @@ -116,10 +106,13 @@ object BoundInMemoryExecutorBenchmark extends HoodieBenchmarkBase { .option("hoodie.insert.shuffle.parallelism", "2") .option("hoodie.datasource.write.operation", "bulk_insert") .option("hoodie.datasource.write.row.writer.enable", "false") - .option("hoodie.bulkinsert.shuffle.parallelism", "10") + .option("hoodie.bulkinsert.shuffle.parallelism", "1") .option("hoodie.upsert.shuffle.parallelism", "2") .option("hoodie.delete.shuffle.parallelism", "2") .option("hoodie.write.executor.type", "DISRUPTOR_EXECUTOR") + .option("hoodie.populate.meta.fields", "false") + .option("hoodie.table.keygenerator.class", "org.apache.hudi.keygen.SimpleKeyGenerator") + .save(new Path(f.getCanonicalPath, finalTableName).toUri.toString) } benchmark.run() From 79dcebea53abcd02839c321dae37d5317ba54c11 Mon Sep 17 00:00:00 2001 From: yuezhang Date: Sun, 24 Apr 2022 18:05:59 +0800 Subject: [PATCH 15/50] add bench mark result --- .../testutils/HoodieExecutorTestUtils.java | 61 ---------- .../benchmark/HoodieExecutorBenchmark.scala | 108 ------------------ .../BoundInMemoryExecutorBenchmark.scala | 12 +- 3 files changed, 8 insertions(+), 173 deletions(-) delete mode 100644 hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/testutils/HoodieExecutorTestUtils.java delete mode 100644 hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/hudi/benchmark/HoodieExecutorBenchmark.scala diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/testutils/HoodieExecutorTestUtils.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/testutils/HoodieExecutorTestUtils.java deleted file mode 100644 index 5528649f945b9..0000000000000 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/testutils/HoodieExecutorTestUtils.java +++ /dev/null @@ -1,61 +0,0 @@ -/* - * 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.testutils; - -import org.apache.avro.Schema; -import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; -import org.apache.hudi.common.testutils.HoodieTestDataGenerator; -import org.apache.hudi.common.util.CollectionUtils; -import org.apache.hudi.common.util.queue.BoundedInMemoryExecutor; -import org.apache.hudi.common.util.queue.BoundedInMemoryQueueConsumer; -import org.apache.hudi.common.util.queue.DisruptorExecutor; -import org.apache.hudi.common.util.queue.WaitStrategyFactory; -import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.execution.HoodieLazyInsertIterable; -import org.apache.spark.TaskContext; -import org.apache.spark.TaskContext$; - -import java.util.List; -import java.util.function.Function; - -public class HoodieExecutorTestUtils { - - public DisruptorExecutor getDisruptorExecutor(List hoodieRecords, - BoundedInMemoryQueueConsumer, Integer> consumer) { - return new DisruptorExecutor(HoodieWriteConfig.WRITE_BUFFER_SIZE.defaultValue(), hoodieRecords.iterator(), consumer, - getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA), WaitStrategyFactory.DEFAULT_STRATEGY, getPreExecuteRunnable()); - } - - public BoundedInMemoryExecutor getBoundedInMemoryExecutor(List hoodieRecords, - BoundedInMemoryQueueConsumer, Integer> consumer) { - return new BoundedInMemoryExecutor(Integer.parseInt(HoodieWriteConfig.WRITE_BUFFER_LIMIT_BYTES_VALUE.defaultValue()), hoodieRecords.iterator(), consumer, - getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA), getPreExecuteRunnable()); - } - - static Function, HoodieLazyInsertIterable.HoodieInsertValueGenResult> getTransformFunction( - Schema schema) { - return hoodieRecord -> new HoodieLazyInsertIterable.HoodieInsertValueGenResult(hoodieRecord, schema, CollectionUtils.EMPTY_PROPERTIES); - } - - private Runnable getPreExecuteRunnable() { - final TaskContext taskContext = TaskContext.get(); - return () -> TaskContext$.MODULE$.setTaskContext(taskContext); - } -} diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/hudi/benchmark/HoodieExecutorBenchmark.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/hudi/benchmark/HoodieExecutorBenchmark.scala deleted file mode 100644 index afec08e0274c4..0000000000000 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/hudi/benchmark/HoodieExecutorBenchmark.scala +++ /dev/null @@ -1,108 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.hudi.benchmark - -import org.apache.hudi.common.model.HoodieRecord -import org.apache.hudi.common.table.timeline.HoodieActiveTimeline -import org.apache.hudi.common.testutils.HoodieTestDataGenerator -import org.apache.hudi.common.util.queue.BoundedInMemoryQueueConsumer -import org.apache.hudi.execution.HoodieLazyInsertIterable -import org.apache.hudi.testutils.HoodieExecutorTestUtils - -object HoodieExecutorBenchmark extends HoodieBenchmarkBase { - - val dataGen = new HoodieTestDataGenerator - - val utils = new HoodieExecutorTestUtils - val recordsNumber = 1000000 - - private def cowTableDisruptorExecutorBenchmark(tableName: String = "executorBenchmark"): Unit = { - val benchmark = new HoodieBenchmark("COW Ingestion", recordsNumber) - benchmark.addCase("Disruptor Executor") { _ => - val con: BoundedInMemoryQueueConsumer[HoodieLazyInsertIterable.HoodieInsertValueGenResult[HoodieRecord[_]], Integer] = new BoundedInMemoryQueueConsumer[HoodieLazyInsertIterable.HoodieInsertValueGenResult[HoodieRecord[_]], Integer]() { - - var count = 0 - /** - * Consumer One record. - */ - override protected def consumeOneRecord(record: HoodieLazyInsertIterable.HoodieInsertValueGenResult[HoodieRecord[_]]): Unit = { - count = count + 1 - } - - /** - * Notifies implementation that we have exhausted consuming records from queue. - */ - override protected def finish(): Unit = { - } - - /** - * Return result of consuming records so far. - */ - override def getResult: Integer = { - count - } - } - val instantTime = HoodieActiveTimeline.createNewInstantTime - val hoodieRecords = dataGen.generateInserts(instantTime, recordsNumber) - val disruptorExecutor = utils.getDisruptorExecutor(hoodieRecords, con) - disruptorExecutor.execute() - disruptorExecutor.shutdownNow() - } - - benchmark.addCase("BoundInMemory Executor") { _ => - val con: BoundedInMemoryQueueConsumer[HoodieLazyInsertIterable.HoodieInsertValueGenResult[HoodieRecord[_]], Integer] = new BoundedInMemoryQueueConsumer[HoodieLazyInsertIterable.HoodieInsertValueGenResult[HoodieRecord[_]], Integer]() { - - var count = 0 - /** - * Consumer One record. - */ - override protected def consumeOneRecord(record: HoodieLazyInsertIterable.HoodieInsertValueGenResult[HoodieRecord[_]]): Unit = { - count = count + 1 - } - - /** - * Notifies implementation that we have exhausted consuming records from queue. - */ - override protected def finish(): Unit = { - } - - /** - * Return result of consuming records so far. - */ - override def getResult: Integer = { - count - } - } - val instantTime = HoodieActiveTimeline.createNewInstantTime - val hoodieRecords = dataGen.generateInserts(instantTime, recordsNumber) - val boundInMemoryExecutor = utils.getBoundedInMemoryExecutor(hoodieRecords, con) - boundInMemoryExecutor.execute() - boundInMemoryExecutor.shutdownNow() - } - - benchmark.run() - } - - override def afterAll(): Unit = { - } - - override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { - cowTableDisruptorExecutorBenchmark() - } -} diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/execution/benchmark/BoundInMemoryExecutorBenchmark.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/execution/benchmark/BoundInMemoryExecutorBenchmark.scala index e5d9c69337683..a5074b61039f6 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/execution/benchmark/BoundInMemoryExecutorBenchmark.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/execution/benchmark/BoundInMemoryExecutorBenchmark.scala @@ -32,7 +32,6 @@ object BoundInMemoryExecutorBenchmark extends HoodieBenchmarkBase { protected val spark: SparkSession = getSparkSession - // 5000 good local val recordNumber = 1000000 def getSparkSession: SparkSession = SparkSession.builder() @@ -66,12 +65,19 @@ object BoundInMemoryExecutorBenchmark extends HoodieBenchmarkBase { spark.createDataFrame(rdd, schema) } + /** + * Intel(R) Xeon(R) Platinum 8259CL CPU @ 2.50GHz + * COW Ingestion: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative + * ------------------------------------------------------------------------------------------------------------------------ + * BoundInMemory Executor 5557 5607 70 0.2 5556.9 1.0X + * Disruptor Executor 2758 2778 28 0.4 2757.7 2.0X + */ private def cowTableDisruptorExecutorBenchmark(tableName: String = "executorBenchmark"): Unit = { + val df = createDataFrame(recordNumber) withTempDir {f => val benchmark = new HoodieBenchmark("COW Ingestion", recordNumber) benchmark.addCase("BoundInMemory Executor") { _ => val finalTableName = tableName + Random.nextInt(10000) - val df = createDataFrame(recordNumber) df.write.format("hudi") .mode(SaveMode.Overwrite) .option("hoodie.datasource.write.recordkey.field", "c1") @@ -88,13 +94,11 @@ object BoundInMemoryExecutorBenchmark extends HoodieBenchmarkBase { .option("hoodie.delete.shuffle.parallelism", "2") .option("hoodie.populate.meta.fields", "false") .option("hoodie.table.keygenerator.class", "org.apache.hudi.keygen.SimpleKeyGenerator") - .save(new Path(f.getCanonicalPath, finalTableName).toUri.toString) } benchmark.addCase("Disruptor Executor") { _ => val finalTableName = tableName + Random.nextInt(10000) - val df = createDataFrame(recordNumber) df.write.format("hudi") .mode(SaveMode.Overwrite) .option("hoodie.datasource.write.recordkey.field", "c1") From e85c9611a700c02a7eff5a0d19621b83988449f1 Mon Sep 17 00:00:00 2001 From: yuezhang Date: Sun, 24 Apr 2022 21:39:10 +0800 Subject: [PATCH 16/50] ready to review --- .../hudi/common/util/queue/DisruptorMessageQueue.java | 10 +--------- 1 file changed, 1 insertion(+), 9 deletions(-) diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageQueue.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageQueue.java index b6d9cdec5f1db..b517ec9061dd4 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageQueue.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageQueue.java @@ -10,9 +10,6 @@ public class DisruptorMessageQueue extends HoodieMessageQueue { - /** Interval used for waiting. **/ - public static final int WAIT_INTERVAL_SEC = 1; - private final Disruptor> queue; private final Function transformFunction; private RingBuffer> ringBuffer; @@ -38,12 +35,7 @@ public long size() { @Override public void insertRecord(I value) throws Exception { - O applied; - if (value == null) { - applied = null; - } else { - applied = transformFunction.apply(value); - } + O applied = transformFunction.apply(value); EventTranslator> translator = new EventTranslator>() { @Override From e00a1093c3e2c4f8201a4cb2ee43fee793780665 Mon Sep 17 00:00:00 2001 From: yuezhang Date: Sun, 24 Apr 2022 22:09:56 +0800 Subject: [PATCH 17/50] code style --- .../org/apache/hudi/common/util/queue/DisruptorExecutor.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorExecutor.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorExecutor.java index bb23a42e604ba..7c5330fb1ee53 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorExecutor.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorExecutor.java @@ -128,7 +128,7 @@ public E execute() { } private void waitForProducersFinished(ExecutorCompletionService pool) throws InterruptedException, ExecutionException { - for (int i = 0; i < producers.size(); i++ ) { + for (int i = 0; i < producers.size(); i++) { pool.take().get(); } } From 74068f6f5ec1dd8bd50903f4d87d4ce61964ae63 Mon Sep 17 00:00:00 2001 From: yuezhang Date: Sun, 24 Apr 2022 22:14:53 +0800 Subject: [PATCH 18/50] code style --- .../apache/hudi/common/util/queue/HoodieDaemonThreadFactory.java | 1 + 1 file changed, 1 insertion(+) diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieDaemonThreadFactory.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieDaemonThreadFactory.java index 3e3daa7513555..ed9b4f528e063 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieDaemonThreadFactory.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieDaemonThreadFactory.java @@ -28,6 +28,7 @@ public class HoodieDaemonThreadFactory implements ThreadFactory { public HoodieDaemonThreadFactory(Runnable preExecuteRunnable) { this.preExecuteRunnable = preExecuteRunnable; } + @Override public Thread newThread(@NotNull final Runnable r) { Thread t = new Thread(new Runnable() { From 03b328ebde347e668e229d645012a16ff84088e2 Mon Sep 17 00:00:00 2001 From: yuezhang Date: Sun, 24 Apr 2022 22:21:04 +0800 Subject: [PATCH 19/50] code style --- .../execution/TestDisruptorMessageQueue.java | 18 ++++++++++++++++++ .../util/queue/DisruptorMessageQueue.java | 18 ++++++++++++++++++ 2 files changed, 36 insertions(+) diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorMessageQueue.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorMessageQueue.java index f7772de4b29d1..c0215878af0e1 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorMessageQueue.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorMessageQueue.java @@ -1,3 +1,21 @@ +/* + * 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.execution; import static org.apache.hudi.execution.HoodieLazyInsertIterable.getTransformFunction; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageQueue.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageQueue.java index b517ec9061dd4..8f6d923fd73c4 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageQueue.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageQueue.java @@ -1,3 +1,21 @@ +/* + * 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; From 3778cf46256d3efb36646b29f06cf149e839045c Mon Sep 17 00:00:00 2001 From: yuezhang Date: Sun, 24 Apr 2022 22:23:56 +0800 Subject: [PATCH 20/50] code style --- .../org/apache/hudi/execution/TestDisruptorExecutionInSpark.java | 1 - 1 file changed, 1 deletion(-) diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorExecutionInSpark.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorExecutionInSpark.java index 8e63206fcf7f1..d66df4d14fe7a 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorExecutionInSpark.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorExecutionInSpark.java @@ -16,7 +16,6 @@ * limitations under the License. */ - package org.apache.hudi.execution; import static org.apache.hudi.execution.HoodieLazyInsertIterable.getTransformFunction; From 1b868643f2f8794b4dd47f8e36124749df6effa3 Mon Sep 17 00:00:00 2001 From: yuezhang Date: Wed, 22 Jun 2022 11:16:31 +0800 Subject: [PATCH 21/50] code review --- .../apache/hudi/config/HoodieWriteConfig.java | 6 +- .../execution/SparkLazyInsertIterable.java | 12 +--- hudi-common/pom.xml | 2 - .../util/queue/DisruptorWaitStrategyType.java | 64 +++++++++++++++++++ .../util/queue/WaitStrategyFactory.java | 29 +++++---- .../BoundInMemoryExecutorBenchmark.scala | 6 +- pom.xml | 7 ++ 7 files changed, 97 insertions(+), 29 deletions(-) create mode 100644 hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorWaitStrategyType.java diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java index 58b6452b57c99..fb124a034e4e1 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java @@ -48,6 +48,7 @@ import org.apache.hudi.common.util.ReflectionUtils; import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.common.util.queue.ExecutorType; import org.apache.hudi.config.metrics.HoodieMetricsCloudWatchConfig; import org.apache.hudi.config.metrics.HoodieMetricsConfig; import org.apache.hudi.config.metrics.HoodieMetricsDatadogConfig; @@ -83,6 +84,7 @@ import java.io.InputStream; import java.util.Arrays; import java.util.List; +import java.util.Locale; import java.util.Map; import java.util.Objects; import java.util.Properties; @@ -988,8 +990,8 @@ public String getKeyGeneratorClass() { return getString(KEYGENERATOR_CLASS_NAME); } - public String getExecutorType() { - return getString(EXECUTOR_TYPE); + public ExecutorType getExecutorType() { + return ExecutorType.valueOf(getString(EXECUTOR_TYPE).toUpperCase(Locale.ROOT)); } public boolean isConsistentLogicalTimestampEnabled() { diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/SparkLazyInsertIterable.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/SparkLazyInsertIterable.java index 98ef5297a7f7e..9f79110ded634 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/SparkLazyInsertIterable.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/SparkLazyInsertIterable.java @@ -36,7 +36,6 @@ import java.util.Iterator; import java.util.List; -import java.util.Locale; public class SparkLazyInsertIterable extends HoodieLazyInsertIterable { @@ -88,16 +87,9 @@ protected List computeNext() { schema = HoodieAvroUtils.addMetadataFields(schema); } - String executorType = hoodieConfig.getExecutorType(); - ExecutorType executorTypeEnum; + ExecutorType executorType = hoodieConfig.getExecutorType(); - try { - executorTypeEnum = ExecutorType.valueOf(executorType.toUpperCase(Locale.ROOT)); - } catch (IllegalArgumentException e) { - throw new HoodieException("Unsupported Executor Type " + executorType); - } - - switch (executorTypeEnum) { + switch (executorType) { case BOUNDED_IN_MEMORY_EXECUTOR: bufferedIteratorExecutor = new BoundedInMemoryExecutor<>(hoodieConfig.getWriteBufferLimitBytes(), inputItr, getInsertHandler(), getTransformFunction(schema, hoodieConfig), hoodieTable.getPreExecuteRunnable()); diff --git a/hudi-common/pom.xml b/hudi-common/pom.xml index 962807b0c852d..2734b97397f97 100644 --- a/hudi-common/pom.xml +++ b/hudi-common/pom.xml @@ -274,9 +274,7 @@ com.lmax disruptor - 3.4.2 - diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorWaitStrategyType.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorWaitStrategyType.java new file mode 100644 index 0000000000000..93a43c415bc1e --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorWaitStrategyType.java @@ -0,0 +1,64 @@ +/* + * 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.apache.hudi.keygen.constant.KeyGeneratorType; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +public enum DisruptorWaitStrategyType { + + /** + * The BlockingWaitStrategy is the slowest of the available wait strategies, but is the most conservative with the respect to CPU usage + * and will give the most consistent behaviour across the widest variety of deployment options. + */ + BLOCKINGWAITSTRATEGY, + + /** + * Like the `BlockingWaitStrategy` the `SleepingWaitStrategy` it attempts to be conservative with CPU usage by using a simple busy wait loop. + * The difference is that the `SleepingWaitStrategy` uses a call to `LockSupport.parkNanos(1)` in the middle of the loop. + * On a typical Linux system this will pause the thread for around 60µs. + */ + SLEEPINGWAITSTRATEGY, + + /** + * The `YieldingWaitStrategy` is one of two WaitStrategies that can be use in low-latency systems. + * It is designed for cases where there is the option to burn CPU cycles with the goal of improving latency. + * The `YieldingWaitStrategy` will busy spin, waiting for the sequence to increment to the appropriate value. + * Inside the body of the loop `Thread#yield()` will be called allowing other queued threads to run. + * This is the recommended wait strategy when you need very high performance, and the number of `EventHandler` threads is lower than the total number of logical cores, + * e.g. you have hyper-threading enabled. + */ + YIELDINGWAITSRATEGY, + + /** + * The `BusySpinWaitStrategy` is the highest performing WaitStrategy. + * Like the `YieldingWaitStrategy`, it can be used in low-latency systems, but puts the highest constraints on the deployment environment. + */ + BUSYSPINWAITSTRATEGY; + + public static List getNames() { + List names = new ArrayList<>(KeyGeneratorType.values().length); + Arrays.stream(KeyGeneratorType.values()) + .forEach(x -> names.add(x.name())); + return names; + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/WaitStrategyFactory.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/WaitStrategyFactory.java index 9492e8b1a88e9..5282101a8612b 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/WaitStrategyFactory.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/WaitStrategyFactory.java @@ -18,6 +18,8 @@ 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; @@ -27,24 +29,25 @@ public class WaitStrategyFactory { - public static final String DEFAULT_STRATEGY = "BlockingWaitStrategy"; + public static final String DEFAULT_STRATEGY = BLOCKINGWAITSTRATEGY.name(); /** * Build WaitStrategy for disruptor */ public static WaitStrategy build(String name) { - WaitStrategy waitStrategy = null; - if ("BlockingWaitStrategy".equals(name)) { - waitStrategy = new BlockingWaitStrategy(); - } else if ("BusySpinWaitStrategy".equals(name)) { - waitStrategy = new BusySpinWaitStrategy(); - } else if ("SleepingWaitStrategy".equals(name)) { - waitStrategy = new SleepingWaitStrategy(); - } else if ("YieldingWaitStrategy".equals(name)) { - waitStrategy = new YieldingWaitStrategy(); - } else { - throw new HoodieException("Invalid wait strategy: " + name); + + DisruptorWaitStrategyType strategyType = DisruptorWaitStrategyType.valueOf(name.toUpperCase()); + switch (strategyType) { + case BLOCKINGWAITSTRATEGY: + return new BlockingWaitStrategy(); + case SLEEPINGWAITSTRATEGY: + return new SleepingWaitStrategy(); + case YIELDINGWAITSRATEGY: + return new YieldingWaitStrategy(); + case BUSYSPINWAITSTRATEGY: + return new BusySpinWaitStrategy(); + default: + throw new HoodieException("Unsupported Executor Type " + name); } - return waitStrategy; } } diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/execution/benchmark/BoundInMemoryExecutorBenchmark.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/execution/benchmark/BoundInMemoryExecutorBenchmark.scala index a5074b61039f6..be9110e1dcf00 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/execution/benchmark/BoundInMemoryExecutorBenchmark.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/execution/benchmark/BoundInMemoryExecutorBenchmark.scala @@ -66,11 +66,13 @@ object BoundInMemoryExecutorBenchmark extends HoodieBenchmarkBase { } /** + * OpenJDK 64-Bit Server VM 1.8.0_161-b14 on Linux 3.10.0-693.21.1.el7.x86_64 * Intel(R) Xeon(R) Platinum 8259CL CPU @ 2.50GHz * COW Ingestion: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative * ------------------------------------------------------------------------------------------------------------------------ - * BoundInMemory Executor 5557 5607 70 0.2 5556.9 1.0X - * Disruptor Executor 2758 2778 28 0.4 2757.7 2.0X + * BoundInMemory Executor 5629 5765 192 0.2 5628.9 1.0X + * Disruptor Executor 2772 2862 127 0.4 2772.2 2.0X + * */ private def cowTableDisruptorExecutorBenchmark(tableName: String = "executorBenchmark"): Unit = { val df = createDataFrame(recordNumber) diff --git a/pom.xml b/pom.xml index e3c8b3e8c6f17..a4d395c6b1d68 100644 --- a/pom.xml +++ b/pom.xml @@ -176,6 +176,7 @@ org.apache.hudi. true 2.7.1 + 3.4.2 4.7 1.12.22 3.17.3 @@ -1069,6 +1070,12 @@ ${zk-curator.version} + + com.lmax + disruptor + ${disruptor.version} + + org.junit.jupiter From 67d8cd3c14c8539a4033c9e4a99b16eef9a03c25 Mon Sep 17 00:00:00 2001 From: yuezhang Date: Wed, 5 Oct 2022 20:42:44 +0800 Subject: [PATCH 22/50] merge from master --- hudi-common/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hudi-common/pom.xml b/hudi-common/pom.xml index c9b8dbef63247..938222b7f4742 100644 --- a/hudi-common/pom.xml +++ b/hudi-common/pom.xml @@ -301,7 +301,7 @@ com.lmax - disruptor + disruptor From b369ccbbba7686ee438d665edb085c010c3628e5 Mon Sep 17 00:00:00 2001 From: yuezhang Date: Thu, 6 Oct 2022 22:07:55 +0800 Subject: [PATCH 23/50] address comments --- .../apache/hudi/config/HoodieWriteConfig.java | 6 +-- .../bootstrap/BootstrapRecordConsumer.java | 4 +- .../table/action/commit/BaseMergeHelper.java | 4 +- .../TestBoundedInMemoryExecutorInSpark.java | 12 ++--- .../execution/TestBoundedInMemoryQueue.java | 4 +- .../TestDisruptorExecutionInSpark.java | 8 ++-- .../execution/TestDisruptorMessageQueue.java | 4 +- .../util/queue/BoundedInMemoryExecutor.java | 23 ++++++---- .../util/queue/BoundedInMemoryQueue.java | 4 +- .../queue/BoundedInMemoryQueueConsumer.java | 11 ++--- .../util/queue/DisruptorBasedProducer.java | 29 ------------ .../common/util/queue/DisruptorExecutor.java | 36 +++++++++------ .../util/queue/DisruptorMessageQueue.java | 19 +++++++- .../common/util/queue/DisruptorPublisher.java | 4 +- .../util/queue/DisruptorWaitStrategyType.java | 8 ++-- .../queue/FunctionBasedDisruptorProducer.java | 42 ----------------- .../queue/FunctionBasedQueueProducer.java | 8 ++-- .../common/util/queue/HoodieConsumer.java | 16 ++----- .../util/queue/HoodieDaemonThreadFactory.java | 10 +++-- .../common/util/queue/HoodieExecutor.java | 44 +++++++++++++++--- .../common/util/queue/HoodieMessageQueue.java | 31 +++++++++++-- .../common/util/queue/HoodieProducer.java | 7 +++ .../queue/IteratorBasedDisruptorProducer.java | 45 ------------------- ...r.java => IteratorBasedQueueConsumer.java} | 40 ++++++++++++++--- .../queue/IteratorBasedQueueProducer.java | 4 +- .../util/queue/WaitStrategyFactory.java | 12 ++--- .../RealtimeUnmergedRecordReader.java | 6 +-- 27 files changed, 220 insertions(+), 221 deletions(-) delete mode 100644 hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorBasedProducer.java delete mode 100644 hudi-common/src/main/java/org/apache/hudi/common/util/queue/FunctionBasedDisruptorProducer.java delete mode 100644 hudi-common/src/main/java/org/apache/hudi/common/util/queue/IteratorBasedDisruptorProducer.java rename hudi-common/src/main/java/org/apache/hudi/common/util/queue/{BoundedInMemoryQueueProducer.java => IteratorBasedQueueConsumer.java} (52%) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java index 6e7eb5d300d94..0b93b06f833f9 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java @@ -244,13 +244,13 @@ public class HoodieWriteConfig extends HoodieConfig { .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") + .key("hoodie.write.executor.disruptor.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 - .key("hoodie.write.wait.strategy") - .defaultValue("BlockingWaitStrategy") + .key("hoodie.write.executor.disruptor.wait.strategy") + .defaultValue("Blocking_Wait_Strategy") .withDocumentation("Strategy employed for making DisruptorExecutor wait on a cursor."); public static final ConfigProperty COMBINE_BEFORE_INSERT = ConfigProperty diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/bootstrap/BootstrapRecordConsumer.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/bootstrap/BootstrapRecordConsumer.java index 8966a5d51c7cb..509df11a52347 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/bootstrap/BootstrapRecordConsumer.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/bootstrap/BootstrapRecordConsumer.java @@ -38,7 +38,7 @@ public BootstrapRecordConsumer(HoodieBootstrapHandle bootstrapHandle) { } @Override - protected void consumeOneRecord(HoodieRecord record) { + public void consumeOneRecord(HoodieRecord record) { try { bootstrapHandle.write(record, ((HoodieRecordPayload) record.getData()) .getInsertValue(bootstrapHandle.getWriterSchemaWithMetaFields())); @@ -48,7 +48,7 @@ protected void consumeOneRecord(HoodieRecord record) { } @Override - protected void finish() {} + public void finish() {} @Override protected Void getResult() { diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseMergeHelper.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseMergeHelper.java index 5ead348140aa3..f750ecedf1a47 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseMergeHelper.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseMergeHelper.java @@ -118,12 +118,12 @@ protected UpdateHandler(HoodieMergeHandle upsertHandle) { } @Override - protected void consumeOneRecord(GenericRecord record) { + public void consumeOneRecord(GenericRecord record) { upsertHandle.write(record); } @Override - protected void finish() {} + public void finish() {} @Override protected Void getResult() { diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestBoundedInMemoryExecutorInSpark.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestBoundedInMemoryExecutorInSpark.java index a714d60d0033a..97f6a7774158b 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestBoundedInMemoryExecutorInSpark.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestBoundedInMemoryExecutorInSpark.java @@ -81,12 +81,12 @@ public void testExecutor() { private int count = 0; @Override - protected void consumeOneRecord(HoodieLazyInsertIterable.HoodieInsertValueGenResult record) { + public void consumeOneRecord(HoodieLazyInsertIterable.HoodieInsertValueGenResult record) { count++; } @Override - protected void finish() { + public void finish() { } @Override @@ -122,7 +122,7 @@ public void testInterruptExecutor() { new BoundedInMemoryQueueConsumer, Integer>() { @Override - protected void consumeOneRecord(HoodieLazyInsertIterable.HoodieInsertValueGenResult record) { + public void consumeOneRecord(HoodieLazyInsertIterable.HoodieInsertValueGenResult record) { try { while (true) { Thread.sleep(1000); @@ -133,7 +133,7 @@ protected void consumeOneRecord(HoodieLazyInsertIterable.HoodieInsertValueGenRes } @Override - protected void finish() { + public void finish() { } @Override @@ -179,11 +179,11 @@ public GenericRecord next() { BoundedInMemoryQueueConsumer, Integer> consumer = new BoundedInMemoryQueueConsumer, Integer>() { @Override - protected void consumeOneRecord(HoodieLazyInsertIterable.HoodieInsertValueGenResult record) { + public void consumeOneRecord(HoodieLazyInsertIterable.HoodieInsertValueGenResult record) { } @Override - protected void finish() { + public void finish() { } @Override diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestBoundedInMemoryQueue.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestBoundedInMemoryQueue.java index 4707a68072e9a..b052f848ea5c8 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestBoundedInMemoryQueue.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestBoundedInMemoryQueue.java @@ -27,8 +27,8 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.SizeEstimator; import org.apache.hudi.common.util.queue.BoundedInMemoryQueue; -import org.apache.hudi.common.util.queue.BoundedInMemoryQueueProducer; import org.apache.hudi.common.util.queue.FunctionBasedQueueProducer; +import org.apache.hudi.common.util.queue.HoodieProducer; import org.apache.hudi.common.util.queue.IteratorBasedQueueProducer; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.testutils.HoodieClientTestHarness; @@ -140,7 +140,7 @@ public void testCompositeProducerRecordReading() throws Exception { recs.add(pRecs); } - List> producers = new ArrayList<>(); + List> producers = new ArrayList<>(); for (int i = 0; i < recs.size(); i++) { final List r = recs.get(i); // Alternate between pull and push based iterators diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorExecutionInSpark.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorExecutionInSpark.java index d66df4d14fe7a..606e3b283b640 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorExecutionInSpark.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorExecutionInSpark.java @@ -84,12 +84,12 @@ public void testExecutor() { private int count = 0; @Override - protected void consumeOneRecord(HoodieLazyInsertIterable.HoodieInsertValueGenResult record) { + public void consumeOneRecord(HoodieLazyInsertIterable.HoodieInsertValueGenResult record) { count++; } @Override - protected void finish() { + public void finish() { } @Override @@ -125,7 +125,7 @@ public void testInterruptExecutor() { new BoundedInMemoryQueueConsumer, Integer>() { @Override - protected void consumeOneRecord(HoodieLazyInsertIterable.HoodieInsertValueGenResult record) { + public void consumeOneRecord(HoodieLazyInsertIterable.HoodieInsertValueGenResult record) { try { while (true) { Thread.sleep(1000); @@ -136,7 +136,7 @@ protected void consumeOneRecord(HoodieLazyInsertIterable.HoodieInsertValueGenRes } @Override - protected void finish() { + public void finish() { } @Override diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorMessageQueue.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorMessageQueue.java index c0215878af0e1..9ce29563d6fef 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorMessageQueue.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorMessageQueue.java @@ -102,7 +102,7 @@ public void testRecordReading() throws Exception { private int count = 0; @Override - protected void consumeOneRecord(HoodieLazyInsertIterable.HoodieInsertValueGenResult record) { + public void consumeOneRecord(HoodieLazyInsertIterable.HoodieInsertValueGenResult record) { count++; afterRecord.add((HoodieAvroRecord) record.record); try { @@ -115,7 +115,7 @@ protected void consumeOneRecord(HoodieLazyInsertIterable.HoodieInsertValueGenRes } @Override - protected void finish() { + public void finish() { } @Override diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryExecutor.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryExecutor.java index 8d20bdc5cca4b..3aa5e6f3fffac 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryExecutor.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryExecutor.java @@ -33,6 +33,7 @@ import java.util.List; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorCompletionService; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; @@ -46,7 +47,7 @@ * class takes as input the size limit, queue producer(s), consumer and transformer and exposes API to orchestrate * concurrent execution of these actors communicating through a central bounded queue */ -public class BoundedInMemoryExecutor extends HoodieExecutor { +public class BoundedInMemoryExecutor implements HoodieExecutor { private static final Logger LOG = LogManager.getLogger(BoundedInMemoryExecutor.class); private static final long TERMINATE_WAITING_TIME_SECS = 60L; @@ -57,7 +58,7 @@ public class BoundedInMemoryExecutor extends HoodieExecutor { // Used for buffering records which is controlled by HoodieWriteConfig#WRITE_BUFFER_LIMIT_BYTES. private final BoundedInMemoryQueue queue; // Producers - private final List> producers; + private final List> producers; // Consumer private final Option> consumer; // pre-execute function to implement environment specific behavior before executors (producers/consumer) run @@ -68,17 +69,17 @@ public BoundedInMemoryExecutor(final long bufferLimitInBytes, final Iterator this(bufferLimitInBytes, new IteratorBasedQueueProducer<>(inputItr), Option.of(consumer), transformFunction, preExecuteRunnable); } - public BoundedInMemoryExecutor(final long bufferLimitInBytes, BoundedInMemoryQueueProducer producer, + public BoundedInMemoryExecutor(final long bufferLimitInBytes, HoodieProducer producer, Option> consumer, final Function transformFunction) { this(bufferLimitInBytes, producer, consumer, transformFunction, Functions.noop()); } - public BoundedInMemoryExecutor(final long bufferLimitInBytes, BoundedInMemoryQueueProducer producer, + public BoundedInMemoryExecutor(final long bufferLimitInBytes, HoodieProducer producer, Option> consumer, final Function transformFunction, Runnable preExecuteRunnable) { this(bufferLimitInBytes, Collections.singletonList(producer), consumer, transformFunction, new DefaultSizeEstimator<>(), preExecuteRunnable); } - public BoundedInMemoryExecutor(final long bufferLimitInBytes, List> producers, + public BoundedInMemoryExecutor(final long bufferLimitInBytes, List> producers, Option> consumer, final Function transformFunction, final SizeEstimator sizeEstimator, Runnable preExecuteRunnable) { this.producers = producers; @@ -126,7 +127,8 @@ public ExecutorCompletionService startProducers() { /** * Start only consumer. */ - private Future startConsumer() { + @Override + public Future startConsumer() { return consumer.map(consumer -> { return consumerExecutorService.submit(() -> { LOG.info("starting consumer thread"); @@ -151,8 +153,7 @@ public E execute() { try { startProducers(); Future future = startConsumer(); - // Wait for consumer to be done - return future.get(); + return finishConsuming(future); } catch (InterruptedException ie) { shutdownNow(); Thread.currentThread().interrupt(); @@ -166,6 +167,12 @@ public boolean isRemaining() { return queue.iterator().hasNext(); } + @Override + public E finishConsuming(Object o) throws ExecutionException, InterruptedException { + Future future = (Future) o; + return future.get(); + } + public void shutdownNow() { producerExecutorService.shutdownNow(); consumerExecutorService.shutdownNow(); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryQueue.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryQueue.java index 4727d0d9b3f01..9f990a3935ed4 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryQueue.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryQueue.java @@ -49,7 +49,7 @@ * @param input payload data type * @param output payload data type */ -public class BoundedInMemoryQueue extends HoodieMessageQueue implements Iterable { +public class BoundedInMemoryQueue implements HoodieMessageQueue { /** Interval used for polling records in the queue. **/ public static final int RECORD_POLL_INTERVAL_SEC = 1; @@ -203,7 +203,7 @@ private boolean expectMoreRecords() { * Reader interface but never exposed to outside world as this is a single consumer queue. Reading is done through a * singleton iterator for this queue. */ - private Option readNextRecord() { + public Option readNextRecord() { if (this.isReadDone.get()) { return Option.empty(); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryQueueConsumer.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryQueueConsumer.java index e8e77e3213d2f..7e7fb885472f7 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryQueueConsumer.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryQueueConsumer.java @@ -23,15 +23,16 @@ /** * Consume entries from queue and execute callback function. */ -public abstract class BoundedInMemoryQueueConsumer extends HoodieConsumer { +public abstract class BoundedInMemoryQueueConsumer implements HoodieConsumer { /** * API to de-queue entries to memory bounded queue. * * @param queue In Memory bounded queue */ - public O consume(BoundedInMemoryQueue queue) throws Exception { - Iterator iterator = queue.iterator(); + @Override + public O consume(HoodieMessageQueue queue) throws Exception { + Iterator iterator = ((BoundedInMemoryQueue) queue).iterator(); while (iterator.hasNext()) { consumeOneRecord(iterator.next()); @@ -46,12 +47,12 @@ public O consume(BoundedInMemoryQueue queue) throws Exception { /** * Consumer One record. */ - protected abstract void consumeOneRecord(I record); + public abstract void consumeOneRecord(I record); /** * Notifies implementation that we have exhausted consuming records from queue. */ - protected abstract void finish(); + public abstract void finish(); /** * Return result of consuming records so far. diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorBasedProducer.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorBasedProducer.java deleted file mode 100644 index 9f7eb7d5d2e10..0000000000000 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorBasedProducer.java +++ /dev/null @@ -1,29 +0,0 @@ -/* - * 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 { - - /** - * API to enqueue entries to memory bounded queue. - * - * @param queue In Memory bounded queue - */ - public abstract void produce(DisruptorMessageQueue queue) throws Exception; -} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorExecutor.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorExecutor.java index e5590b7fbb1b1..dcb2d4224a6fa 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorExecutor.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorExecutor.java @@ -34,10 +34,11 @@ import java.util.concurrent.ExecutorCompletionService; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; +import java.util.concurrent.Future; import java.util.function.Function; import java.util.stream.Collectors; -public class DisruptorExecutor extends HoodieExecutor { +public class DisruptorExecutor implements HoodieExecutor { private static final Logger LOG = LogManager.getLogger(DisruptorExecutor.class); @@ -48,7 +49,7 @@ public class DisruptorExecutor extends HoodieExecutor { // Used for buffering records which is controlled by HoodieWriteConfig#WRITE_BUFFER_LIMIT_BYTES. private final DisruptorMessageQueue queue; // Producers - private final List> producers; + private final List> producers; // Consumer private final Option> consumer; // pre-execute function to implement environment specific behavior before executors (producers/consumer) run @@ -56,20 +57,20 @@ public class DisruptorExecutor extends HoodieExecutor { 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); + this(bufferSize, new IteratorBasedQueueProducer<>(inputItr), Option.of(consumer), transformFunction, waitStrategy, preExecuteRunnable); } - public DisruptorExecutor(final int bufferSize, DisruptorBasedProducer producer, + public DisruptorExecutor(final int bufferSize, HoodieProducer producer, Option> consumer, final Function transformFunction) { this(bufferSize, producer, consumer, transformFunction, WaitStrategyFactory.DEFAULT_STRATEGY, Functions.noop()); } - public DisruptorExecutor(final int bufferSize, DisruptorBasedProducer producer, + public DisruptorExecutor(final int bufferSize, HoodieProducer 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, + public DisruptorExecutor(final int bufferSize, List> producers, Option> consumer, final Function transformFunction, final String waitStrategy, Runnable preExecuteRunnable) { this.producers = producers; @@ -110,14 +111,9 @@ public ExecutorCompletionService startProducers() { public E execute() { try { assert consumer.isPresent(); - setupConsumer(); + startConsumer(); ExecutorCompletionService pool = startProducers(); - - waitForProducersFinished(pool); - queue.getInnerQueue().shutdown(); - consumer.get().finish(); - - return consumer.get().getResult(); + return finishConsuming(pool); } catch (InterruptedException ie) { shutdownNow(); Thread.currentThread().interrupt(); @@ -127,6 +123,16 @@ public E execute() { } } + @Override + public E finishConsuming(Object o) throws ExecutionException, InterruptedException { + ExecutorCompletionService pool = (ExecutorCompletionService) o; + waitForProducersFinished(pool); + queue.getInnerQueue().shutdown(); + consumer.get().finish(); + + return consumer.get().getResult(); + } + private void waitForProducersFinished(ExecutorCompletionService pool) throws InterruptedException, ExecutionException { for (int i = 0; i < producers.size(); i++) { pool.take().get(); @@ -136,12 +142,14 @@ private void waitForProducersFinished(ExecutorCompletionService pool) t /** * Start only consumer. */ - private void setupConsumer() { + @Override + public Future startConsumer() { DisruptorMessageHandler handler = new DisruptorMessageHandler<>(consumer.get()); Disruptor> innerQueue = queue.getInnerQueue(); innerQueue.handleEventsWith(handler); innerQueue.start(); + return null; } @Override diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageQueue.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageQueue.java index 8f6d923fd73c4..9b01b24a6fb75 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageQueue.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageQueue.java @@ -23,10 +23,13 @@ import com.lmax.disruptor.WaitStrategy; import com.lmax.disruptor.dsl.Disruptor; import com.lmax.disruptor.dsl.ProducerType; +import org.apache.hudi.common.util.Option; +import java.util.ArrayList; +import java.util.Iterator; import java.util.function.Function; -public class DisruptorMessageQueue extends HoodieMessageQueue { +public class DisruptorMessageQueue implements HoodieMessageQueue { private final Disruptor> queue; private final Function transformFunction; @@ -65,6 +68,12 @@ public void translateTo(HoodieDisruptorEvent event, long sequence) { queue.getRingBuffer().publishEvent(translator); } + @Override + public Option readNextRecord() { + // Let DisruptorMessageHandler to handle consuming logic. + return null; + } + @Override public void close() { queue.shutdown(); @@ -77,4 +86,12 @@ public Disruptor> getInnerQueue() { public boolean isEmpty() { return ringBuffer.getBufferSize() == ringBuffer.remainingCapacity(); } + + @Override + public Iterator iterator() { + // fake a dummy iterator, because disruptor engine can take care of iterating consuming records. + ArrayList dummyList = new ArrayList<>(); + dummyList.add(ringBuffer.get(0).get()); + return dummyList.iterator(); + } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorPublisher.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorPublisher.java index fe3378fa1d487..accc633772765 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorPublisher.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorPublisher.java @@ -20,10 +20,10 @@ public class DisruptorPublisher { - DisruptorBasedProducer producer; + HoodieProducer producer; private final DisruptorMessageQueue queue; - public DisruptorPublisher(DisruptorBasedProducer producer, DisruptorMessageQueue queue) { + public DisruptorPublisher(HoodieProducer producer, DisruptorMessageQueue queue) { this.producer = producer; this.queue = queue; } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorWaitStrategyType.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorWaitStrategyType.java index 93a43c415bc1e..45c086cda9ad3 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorWaitStrategyType.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorWaitStrategyType.java @@ -30,14 +30,14 @@ public enum DisruptorWaitStrategyType { * The BlockingWaitStrategy is the slowest of the available wait strategies, but is the most conservative with the respect to CPU usage * and will give the most consistent behaviour across the widest variety of deployment options. */ - BLOCKINGWAITSTRATEGY, + BLOCKING_WAIT_STRATEGY, /** * Like the `BlockingWaitStrategy` the `SleepingWaitStrategy` it attempts to be conservative with CPU usage by using a simple busy wait loop. * The difference is that the `SleepingWaitStrategy` uses a call to `LockSupport.parkNanos(1)` in the middle of the loop. * On a typical Linux system this will pause the thread for around 60µs. */ - SLEEPINGWAITSTRATEGY, + SLEEPING_WAIT_STRATEGY, /** * The `YieldingWaitStrategy` is one of two WaitStrategies that can be use in low-latency systems. @@ -47,13 +47,13 @@ public enum DisruptorWaitStrategyType { * This is the recommended wait strategy when you need very high performance, and the number of `EventHandler` threads is lower than the total number of logical cores, * e.g. you have hyper-threading enabled. */ - YIELDINGWAITSRATEGY, + YIELDING_WAIT_SRATEGY, /** * The `BusySpinWaitStrategy` is the highest performing WaitStrategy. * Like the `YieldingWaitStrategy`, it can be used in low-latency systems, but puts the highest constraints on the deployment environment. */ - BUSYSPINWAITSTRATEGY; + BUSY_SPIN_WAIT_STRATEGY; public static List getNames() { List names = new ArrayList<>(KeyGeneratorType.values().length); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/FunctionBasedDisruptorProducer.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/FunctionBasedDisruptorProducer.java deleted file mode 100644 index 86538687e7c8d..0000000000000 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/FunctionBasedDisruptorProducer.java +++ /dev/null @@ -1,42 +0,0 @@ -/* - * 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.apache.log4j.LogManager; -import org.apache.log4j.Logger; - -import java.util.function.Function; - -public class FunctionBasedDisruptorProducer extends DisruptorBasedProducer { - - private static final Logger LOG = LogManager.getLogger(FunctionBasedDisruptorProducer.class); - - private final Function, Boolean> producerFunction; - - public FunctionBasedDisruptorProducer(Function, Boolean> producerFunction) { - this.producerFunction = producerFunction; - } - - @Override - public void produce(DisruptorMessageQueue queue) throws Exception { - LOG.info("starting function which will enqueue records"); - producerFunction.apply(queue); - LOG.info("finished function which will enqueue records"); - } -} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/FunctionBasedQueueProducer.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/FunctionBasedQueueProducer.java index d391c0e8c6192..df199158784bb 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/FunctionBasedQueueProducer.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/FunctionBasedQueueProducer.java @@ -28,18 +28,18 @@ * * @param Type of entry produced for queue */ -public class FunctionBasedQueueProducer extends BoundedInMemoryQueueProducer { +public class FunctionBasedQueueProducer implements HoodieProducer { private static final Logger LOG = LogManager.getLogger(FunctionBasedQueueProducer.class); - private final Function, Boolean> producerFunction; + private final Function, Boolean> producerFunction; - public FunctionBasedQueueProducer(Function, Boolean> producerFunction) { + public FunctionBasedQueueProducer(Function, Boolean> producerFunction) { this.producerFunction = producerFunction; } @Override - public void produce(BoundedInMemoryQueue queue) { + public void produce(HoodieMessageQueue queue) { LOG.info("starting function which will enqueue records"); producerFunction.apply(queue); LOG.info("finished function which will enqueue records"); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieConsumer.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieConsumer.java index e308f41c8da78..0c5f484fcd68b 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieConsumer.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieConsumer.java @@ -18,20 +18,10 @@ package org.apache.hudi.common.util.queue; -public abstract class HoodieConsumer { +public interface HoodieConsumer { /** - * Consumer One record. + * Consume records from inner message queue. */ - 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(); + O consume(HoodieMessageQueue queue) throws Exception; } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieDaemonThreadFactory.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieDaemonThreadFactory.java index ed9b4f528e063..77f0eb6a0d888 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieDaemonThreadFactory.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieDaemonThreadFactory.java @@ -18,19 +18,23 @@ package org.apache.hudi.common.util.queue; -import org.jetbrains.annotations.NotNull; import java.util.concurrent.ThreadFactory; +import java.util.concurrent.atomic.AtomicInteger; public class HoodieDaemonThreadFactory implements ThreadFactory { private Runnable preExecuteRunnable; + private final AtomicInteger threadsNum = new AtomicInteger(); + private final String namePattern; + private final String baseName = "Hoodie-disruptor-daemon-thread"; public HoodieDaemonThreadFactory(Runnable preExecuteRunnable) { this.preExecuteRunnable = preExecuteRunnable; + this.namePattern = baseName + "-%d"; } @Override - public Thread newThread(@NotNull final Runnable r) { + public Thread newThread(Runnable r) { Thread t = new Thread(new Runnable() { @Override @@ -38,7 +42,7 @@ public void run() { preExecuteRunnable.run(); r.run(); } - }); + }, String.format(namePattern, threadsNum.addAndGet(1))); t.setDaemon(true); return t; } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieExecutor.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieExecutor.java index fdf83d409aa1a..4e1fccb2c6df5 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieExecutor.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieExecutor.java @@ -18,19 +18,49 @@ package org.apache.hudi.common.util.queue; +import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorCompletionService; +import java.util.concurrent.Future; -public abstract class HoodieExecutor { +/** + * HoodieExecutor which orchestrates concurrent producers and consumers communicating through a bounded in message queue. + */ +public interface HoodieExecutor { + + /** + * Start all Producers. + */ + ExecutorCompletionService startProducers(); + + /** + * Start consumer. + */ + Future startConsumer(); - public abstract ExecutorCompletionService startProducers(); + /** + * Main API to + * 1. Set up and run all the production + * 2. Set up and run all the consumption. + * 3. Shutdown and return the result. + */ + E execute(); - public abstract E execute(); + boolean isRemaining(); - public abstract boolean isRemaining(); + /** + * Wait for consumer finish consuming and return result. + */ + E finishConsuming(Object o) throws ExecutionException, InterruptedException; - public abstract void shutdownNow(); + /** + * Shutdown all the consumers and producers. + */ + void shutdownNow(); - public abstract HoodieMessageQueue getQueue(); + /** + * get bounded in message queue. + */ + HoodieMessageQueue getQueue(); - public abstract boolean awaitTermination(); + boolean awaitTermination(); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieMessageQueue.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieMessageQueue.java index fe7848b1e09bc..0e6dde13b043a 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieMessageQueue.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieMessageQueue.java @@ -18,11 +18,34 @@ package org.apache.hudi.common.util.queue; -public abstract class HoodieMessageQueue { +import org.apache.hudi.common.util.Option; - public abstract long size(); +import java.util.Iterator; - public abstract void insertRecord(I t) throws Exception; +/** + * HoodieMessageQueue holds an internal message queue, and control the behavior of + * 1. insert record into internal message queue. + * 2. get record from internal message queue. + * 3. close internal message queue. + */ +public interface HoodieMessageQueue extends Iterable{ + + /** + * 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(); - public abstract void close(); + Iterator iterator(); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieProducer.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieProducer.java index 736f9b93fadbd..d9f6001abecdb 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieProducer.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieProducer.java @@ -19,4 +19,11 @@ package org.apache.hudi.common.util.queue; public interface HoodieProducer { + + /** + * API to enqueue entries to bounded queue. + * + * @param queue In Memory bounded queue + */ + void produce(HoodieMessageQueue queue) throws Exception; } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/IteratorBasedDisruptorProducer.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/IteratorBasedDisruptorProducer.java deleted file mode 100644 index 23bc7705d3f03..0000000000000 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/IteratorBasedDisruptorProducer.java +++ /dev/null @@ -1,45 +0,0 @@ -/* - * 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.apache.log4j.LogManager; -import org.apache.log4j.Logger; - -import java.util.Iterator; - -public class IteratorBasedDisruptorProducer extends DisruptorBasedProducer { - - private static final Logger LOG = LogManager.getLogger(IteratorBasedDisruptorProducer.class); - - // input iterator for producing items in the buffer. - private final Iterator inputIterator; - - public IteratorBasedDisruptorProducer(Iterator inputIterator) { - this.inputIterator = inputIterator; - } - - @Override - public void produce(DisruptorMessageQueue queue) throws Exception { - LOG.info("starting to buffer records"); - while (inputIterator.hasNext()) { - queue.insertRecord(inputIterator.next()); - } - LOG.info("finished buffering records"); - } -} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryQueueProducer.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/IteratorBasedQueueConsumer.java similarity index 52% rename from hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryQueueProducer.java rename to hudi-common/src/main/java/org/apache/hudi/common/util/queue/IteratorBasedQueueConsumer.java index 945416de169da..1b6c8a93494f6 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryQueueProducer.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/IteratorBasedQueueConsumer.java @@ -18,17 +18,45 @@ package org.apache.hudi.common.util.queue; +import java.util.Iterator; + /** - * Producer for {@link BoundedInMemoryQueue}. Memory Bounded Buffer supports multiple producers single consumer pattern. - * - * @param Input type for buffer items produced + * Consume entries from queue and execute callback function. */ -public abstract class BoundedInMemoryQueueProducer implements HoodieProducer { +public abstract class IteratorBasedQueueConsumer implements HoodieConsumer { /** - * API to enqueue entries to memory bounded queue. + * API to de-queue entries to memory bounded queue. * * @param queue In Memory bounded queue */ - public abstract void produce(BoundedInMemoryQueue queue) throws Exception; + @Override + public O consume(HoodieMessageQueue queue) throws Exception { + Iterator iterator = queue.iterator(); + + while (iterator.hasNext()) { + consumeOneRecord(iterator.next()); + } + + // Notifies done + finish(); + + return getResult(); + } + + /** + * Consumer One record. + */ + public abstract void consumeOneRecord(I record); + + /** + * Notifies implementation that we have exhausted consuming records from queue. + */ + public abstract void finish(); + + /** + * Return result of consuming records so far. + */ + protected abstract O getResult(); + } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/IteratorBasedQueueProducer.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/IteratorBasedQueueProducer.java index 8459e7770d59e..7904fd61ebc80 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/IteratorBasedQueueProducer.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/IteratorBasedQueueProducer.java @@ -28,7 +28,7 @@ * * @param Item type produced for the buffer. */ -public class IteratorBasedQueueProducer extends BoundedInMemoryQueueProducer { +public class IteratorBasedQueueProducer implements HoodieProducer { private static final Logger LOG = LogManager.getLogger(IteratorBasedQueueProducer.class); @@ -40,7 +40,7 @@ public IteratorBasedQueueProducer(Iterator inputIterator) { } @Override - public void produce(BoundedInMemoryQueue queue) throws Exception { + public void produce(HoodieMessageQueue queue) throws Exception { LOG.info("starting to buffer records"); while (inputIterator.hasNext()) { queue.insertRecord(inputIterator.next()); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/WaitStrategyFactory.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/WaitStrategyFactory.java index 5282101a8612b..c69dc3859dc0a 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/WaitStrategyFactory.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/WaitStrategyFactory.java @@ -18,7 +18,7 @@ package org.apache.hudi.common.util.queue; -import static org.apache.hudi.common.util.queue.DisruptorWaitStrategyType.BLOCKINGWAITSTRATEGY; +import static org.apache.hudi.common.util.queue.DisruptorWaitStrategyType.BLOCKING_WAIT_STRATEGY; import com.lmax.disruptor.BlockingWaitStrategy; import com.lmax.disruptor.BusySpinWaitStrategy; @@ -29,7 +29,7 @@ public class WaitStrategyFactory { - public static final String DEFAULT_STRATEGY = BLOCKINGWAITSTRATEGY.name(); + public static final String DEFAULT_STRATEGY = BLOCKING_WAIT_STRATEGY.name(); /** * Build WaitStrategy for disruptor @@ -38,13 +38,13 @@ public static WaitStrategy build(String name) { DisruptorWaitStrategyType strategyType = DisruptorWaitStrategyType.valueOf(name.toUpperCase()); switch (strategyType) { - case BLOCKINGWAITSTRATEGY: + case BLOCKING_WAIT_STRATEGY: return new BlockingWaitStrategy(); - case SLEEPINGWAITSTRATEGY: + case SLEEPING_WAIT_STRATEGY: return new SleepingWaitStrategy(); - case YIELDINGWAITSRATEGY: + case YIELDING_WAIT_SRATEGY: return new YieldingWaitStrategy(); - case BUSYSPINWAITSTRATEGY: + case BUSY_SPIN_WAIT_STRATEGY: return new BusySpinWaitStrategy(); default: throw new HoodieException("Unsupported Executor Type " + name); diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeUnmergedRecordReader.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeUnmergedRecordReader.java index 84c808865072a..700a87cbb7284 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeUnmergedRecordReader.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeUnmergedRecordReader.java @@ -24,8 +24,8 @@ import org.apache.hudi.common.util.Functions; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.queue.BoundedInMemoryExecutor; -import org.apache.hudi.common.util.queue.BoundedInMemoryQueueProducer; import org.apache.hudi.common.util.queue.FunctionBasedQueueProducer; +import org.apache.hudi.common.util.queue.HoodieProducer; import org.apache.hudi.common.util.queue.IteratorBasedQueueProducer; import org.apache.hudi.hadoop.RecordReaderValueIterator; import org.apache.hudi.hadoop.SafeParquetRecordReaderWrapper; @@ -104,8 +104,8 @@ public RealtimeUnmergedRecordReader(RealtimeSplit split, JobConf job, /** * Setup log and parquet reading in parallel. Both write to central buffer. */ - private List> getParallelProducers() { - List> producers = new ArrayList<>(); + private List> getParallelProducers() { + List> producers = new ArrayList<>(); producers.add(new FunctionBasedQueueProducer<>(buffer -> { logRecordScanner.scan(); return null; From 92760dbf5a047fe1f9941fa4b36c944eb3bec5c7 Mon Sep 17 00:00:00 2001 From: yuezhang Date: Thu, 6 Oct 2022 23:06:21 +0800 Subject: [PATCH 24/50] address comments --- .../execution/CopyOnWriteInsertHandler.java | 4 +- .../bootstrap/BootstrapRecordConsumer.java | 4 +- .../table/action/commit/BaseMergeHelper.java | 4 +- .../hudi/execution/ExplicitWriteHandler.java | 4 +- .../TestBoundedInMemoryExecutorInSpark.java | 14 ++--- .../TestDisruptorExecutionInSpark.java | 10 +-- .../execution/TestDisruptorMessageQueue.java | 6 +- .../view/AbstractTableFileSystemView.java | 2 +- .../util/queue/BoundedInMemoryExecutor.java | 10 +-- .../queue/BoundedInMemoryQueueConsumer.java | 62 ------------------- .../common/util/queue/DisruptorExecutor.java | 10 +-- .../util/queue/DisruptorMessageHandler.java | 4 +- .../common/util/queue/HoodieProducer.java | 5 ++ .../apache/hudi/table/format/FormatUtils.java | 6 +- 14 files changed, 44 insertions(+), 101 deletions(-) delete mode 100644 hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryQueueConsumer.java diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/execution/CopyOnWriteInsertHandler.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/execution/CopyOnWriteInsertHandler.java index 5e1f832b7f239..abb1122289453 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/execution/CopyOnWriteInsertHandler.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/execution/CopyOnWriteInsertHandler.java @@ -22,7 +22,7 @@ import org.apache.hudi.common.engine.TaskContextSupplier; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; -import org.apache.hudi.common.util.queue.BoundedInMemoryQueueConsumer; +import org.apache.hudi.common.util.queue.IteratorBasedQueueConsumer; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.execution.HoodieLazyInsertIterable.HoodieInsertValueGenResult; import org.apache.hudi.io.HoodieWriteHandle; @@ -38,7 +38,7 @@ * Consumes stream of hoodie records from in-memory queue and writes to one or more create-handles. */ public class CopyOnWriteInsertHandler - extends BoundedInMemoryQueueConsumer, List> { + extends IteratorBasedQueueConsumer, List> { private HoodieWriteConfig config; private String instantTime; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/bootstrap/BootstrapRecordConsumer.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/bootstrap/BootstrapRecordConsumer.java index 509df11a52347..76968c6108d96 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/bootstrap/BootstrapRecordConsumer.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/bootstrap/BootstrapRecordConsumer.java @@ -20,7 +20,7 @@ import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; -import org.apache.hudi.common.util.queue.BoundedInMemoryQueueConsumer; +import org.apache.hudi.common.util.queue.IteratorBasedQueueConsumer; import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.io.HoodieBootstrapHandle; @@ -29,7 +29,7 @@ /** * Consumer that dequeues records from queue and sends to Merge Handle for writing. */ -public class BootstrapRecordConsumer extends BoundedInMemoryQueueConsumer { +public class BootstrapRecordConsumer extends IteratorBasedQueueConsumer { private final HoodieBootstrapHandle bootstrapHandle; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseMergeHelper.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseMergeHelper.java index f750ecedf1a47..bd1c01958bb61 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseMergeHelper.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseMergeHelper.java @@ -22,7 +22,7 @@ import org.apache.hudi.client.utils.MergingIterator; import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieRecordPayload; -import org.apache.hudi.common.util.queue.BoundedInMemoryQueueConsumer; +import org.apache.hudi.common.util.queue.IteratorBasedQueueConsumer; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.io.HoodieMergeHandle; import org.apache.hudi.io.storage.HoodieFileReader; @@ -109,7 +109,7 @@ protected Iterator getMergingIterator(HoodieTable tab /** * Consumer that dequeues records from queue and sends to Merge Handle. */ - protected static class UpdateHandler extends BoundedInMemoryQueueConsumer { + protected static class UpdateHandler extends IteratorBasedQueueConsumer { private final HoodieMergeHandle upsertHandle; diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/execution/ExplicitWriteHandler.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/execution/ExplicitWriteHandler.java index 46eff587575cc..9dab2170e09c1 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/execution/ExplicitWriteHandler.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/execution/ExplicitWriteHandler.java @@ -21,7 +21,7 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; -import org.apache.hudi.common.util.queue.BoundedInMemoryQueueConsumer; +import org.apache.hudi.common.util.queue.IteratorBasedQueueConsumer; import org.apache.hudi.io.HoodieWriteHandle; import java.util.ArrayList; @@ -31,7 +31,7 @@ * Consumes stream of hoodie records from in-memory queue and writes to one explicit create handle. */ public class ExplicitWriteHandler - extends BoundedInMemoryQueueConsumer, List> { + extends IteratorBasedQueueConsumer, List> { private final List statuses = new ArrayList<>(); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestBoundedInMemoryExecutorInSpark.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestBoundedInMemoryExecutorInSpark.java index 97f6a7774158b..69457428d107e 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestBoundedInMemoryExecutorInSpark.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestBoundedInMemoryExecutorInSpark.java @@ -23,7 +23,7 @@ import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.queue.BoundedInMemoryExecutor; -import org.apache.hudi.common.util.queue.BoundedInMemoryQueueConsumer; +import org.apache.hudi.common.util.queue.IteratorBasedQueueConsumer; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.testutils.HoodieClientTestHarness; @@ -75,8 +75,8 @@ public void testExecutor() { HoodieWriteConfig hoodieWriteConfig = mock(HoodieWriteConfig.class); when(hoodieWriteConfig.getWriteBufferLimitBytes()).thenReturn(1024); - BoundedInMemoryQueueConsumer, Integer> consumer = - new BoundedInMemoryQueueConsumer, Integer>() { + IteratorBasedQueueConsumer, Integer> consumer = + new IteratorBasedQueueConsumer, Integer>() { private int count = 0; @@ -118,8 +118,8 @@ public void testInterruptExecutor() { HoodieWriteConfig hoodieWriteConfig = mock(HoodieWriteConfig.class); when(hoodieWriteConfig.getWriteBufferLimitBytes()).thenReturn(1024); - BoundedInMemoryQueueConsumer, Integer> consumer = - new BoundedInMemoryQueueConsumer, Integer>() { + IteratorBasedQueueConsumer, Integer> consumer = + new IteratorBasedQueueConsumer, Integer>() { @Override public void consumeOneRecord(HoodieLazyInsertIterable.HoodieInsertValueGenResult record) { @@ -176,8 +176,8 @@ public GenericRecord next() { } }; - BoundedInMemoryQueueConsumer, Integer> consumer = - new BoundedInMemoryQueueConsumer, Integer>() { + IteratorBasedQueueConsumer, Integer> consumer = + new IteratorBasedQueueConsumer, Integer>() { @Override public void consumeOneRecord(HoodieLazyInsertIterable.HoodieInsertValueGenResult record) { } diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorExecutionInSpark.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorExecutionInSpark.java index 606e3b283b640..be057e64056ca 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorExecutionInSpark.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorExecutionInSpark.java @@ -25,7 +25,7 @@ import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.common.util.Option; -import org.apache.hudi.common.util.queue.BoundedInMemoryQueueConsumer; +import org.apache.hudi.common.util.queue.IteratorBasedQueueConsumer; import org.apache.hudi.common.util.queue.DisruptorExecutor; import org.apache.hudi.common.util.queue.WaitStrategyFactory; import org.apache.hudi.config.HoodieWriteConfig; @@ -78,8 +78,8 @@ public void testExecutor() { HoodieWriteConfig hoodieWriteConfig = mock(HoodieWriteConfig.class); when(hoodieWriteConfig.getWriteBufferSize()).thenReturn(8); - BoundedInMemoryQueueConsumer, Integer> consumer = - new BoundedInMemoryQueueConsumer, Integer>() { + IteratorBasedQueueConsumer, Integer> consumer = + new IteratorBasedQueueConsumer, Integer>() { private int count = 0; @@ -121,8 +121,8 @@ public void testInterruptExecutor() { HoodieWriteConfig hoodieWriteConfig = mock(HoodieWriteConfig.class); when(hoodieWriteConfig.getWriteBufferSize()).thenReturn(1024); - BoundedInMemoryQueueConsumer, Integer> consumer = - new BoundedInMemoryQueueConsumer, Integer>() { + IteratorBasedQueueConsumer, Integer> consumer = + new IteratorBasedQueueConsumer, Integer>() { @Override public void consumeOneRecord(HoodieLazyInsertIterable.HoodieInsertValueGenResult record) { diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorMessageQueue.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorMessageQueue.java index 9ce29563d6fef..3487cd8317394 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorMessageQueue.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorMessageQueue.java @@ -26,7 +26,7 @@ import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.common.util.Option; -import org.apache.hudi.common.util.queue.BoundedInMemoryQueueConsumer; +import org.apache.hudi.common.util.queue.IteratorBasedQueueConsumer; import org.apache.hudi.common.util.queue.DisruptorExecutor; import org.apache.hudi.common.util.queue.WaitStrategyFactory; import org.apache.hudi.config.HoodieWriteConfig; @@ -96,8 +96,8 @@ public void testRecordReading() throws Exception { HoodieWriteConfig hoodieWriteConfig = mock(HoodieWriteConfig.class); when(hoodieWriteConfig.getWriteBufferSize()).thenReturn(16); - BoundedInMemoryQueueConsumer, Integer> consumer = - new BoundedInMemoryQueueConsumer, Integer>() { + IteratorBasedQueueConsumer, Integer> consumer = + new IteratorBasedQueueConsumer, Integer>() { private int count = 0; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/AbstractTableFileSystemView.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/AbstractTableFileSystemView.java index 976217ae07b78..d97127235bb89 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/view/AbstractTableFileSystemView.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/AbstractTableFileSystemView.java @@ -1,5 +1,5 @@ /* - * Licensed to the Apache Software Foundation (ASF) under one + * LicresolveCommitConflictnsed 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 diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryExecutor.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryExecutor.java index 3aa5e6f3fffac..b5fe96f9a641a 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryExecutor.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryExecutor.java @@ -60,27 +60,27 @@ public class BoundedInMemoryExecutor implements HoodieExecutor // Producers private final List> producers; // Consumer - private final Option> consumer; + private final Option> consumer; // pre-execute function to implement environment specific behavior before executors (producers/consumer) run private final Runnable preExecuteRunnable; public BoundedInMemoryExecutor(final long bufferLimitInBytes, final Iterator inputItr, - BoundedInMemoryQueueConsumer consumer, Function transformFunction, Runnable preExecuteRunnable) { + IteratorBasedQueueConsumer consumer, Function transformFunction, Runnable preExecuteRunnable) { this(bufferLimitInBytes, new IteratorBasedQueueProducer<>(inputItr), Option.of(consumer), transformFunction, preExecuteRunnable); } public BoundedInMemoryExecutor(final long bufferLimitInBytes, HoodieProducer producer, - Option> consumer, final Function transformFunction) { + Option> consumer, final Function transformFunction) { this(bufferLimitInBytes, producer, consumer, transformFunction, Functions.noop()); } public BoundedInMemoryExecutor(final long bufferLimitInBytes, HoodieProducer producer, - Option> consumer, final Function transformFunction, Runnable preExecuteRunnable) { + Option> consumer, final Function transformFunction, Runnable preExecuteRunnable) { this(bufferLimitInBytes, Collections.singletonList(producer), consumer, transformFunction, new DefaultSizeEstimator<>(), preExecuteRunnable); } public BoundedInMemoryExecutor(final long bufferLimitInBytes, List> producers, - Option> consumer, final Function transformFunction, + Option> consumer, final Function transformFunction, final SizeEstimator sizeEstimator, Runnable preExecuteRunnable) { this.producers = producers; this.consumer = consumer; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryQueueConsumer.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryQueueConsumer.java deleted file mode 100644 index 7e7fb885472f7..0000000000000 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryQueueConsumer.java +++ /dev/null @@ -1,62 +0,0 @@ -/* - * 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.Iterator; - -/** - * Consume entries from queue and execute callback function. - */ -public abstract class BoundedInMemoryQueueConsumer implements HoodieConsumer { - - /** - * API to de-queue entries to memory bounded queue. - * - * @param queue In Memory bounded queue - */ - @Override - public O consume(HoodieMessageQueue queue) throws Exception { - Iterator iterator = ((BoundedInMemoryQueue) queue).iterator(); - - while (iterator.hasNext()) { - consumeOneRecord(iterator.next()); - } - - // Notifies done - finish(); - - return getResult(); - } - - /** - * Consumer One record. - */ - public abstract void consumeOneRecord(I record); - - /** - * Notifies implementation that we have exhausted consuming records from queue. - */ - public abstract void finish(); - - /** - * Return result of consuming records so far. - */ - protected abstract O getResult(); - -} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorExecutor.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorExecutor.java index dcb2d4224a6fa..70159c0cdc506 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorExecutor.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorExecutor.java @@ -51,27 +51,27 @@ public class DisruptorExecutor implements HoodieExecutor { // Producers private final List> producers; // Consumer - private final Option> 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) { + IteratorBasedQueueConsumer consumer, Function transformFunction, String waitStrategy, Runnable preExecuteRunnable) { this(bufferSize, new IteratorBasedQueueProducer<>(inputItr), Option.of(consumer), transformFunction, waitStrategy, preExecuteRunnable); } public DisruptorExecutor(final int bufferSize, HoodieProducer producer, - Option> consumer, final Function transformFunction) { + Option> consumer, final Function transformFunction) { this(bufferSize, producer, consumer, transformFunction, WaitStrategyFactory.DEFAULT_STRATEGY, Functions.noop()); } public DisruptorExecutor(final int bufferSize, HoodieProducer producer, - Option> consumer, final Function transformFunction, String waitStrategy, Runnable preExecuteRunnable) { + 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, + Option> consumer, final Function transformFunction, final String waitStrategy, Runnable preExecuteRunnable) { this.producers = producers; this.consumer = consumer; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageHandler.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageHandler.java index 47308cba86007..f98108224c9d2 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageHandler.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageHandler.java @@ -22,9 +22,9 @@ public class DisruptorMessageHandler implements EventHandler> { - private BoundedInMemoryQueueConsumer consumer; + private IteratorBasedQueueConsumer consumer; - public DisruptorMessageHandler(BoundedInMemoryQueueConsumer consumer) { + public DisruptorMessageHandler(IteratorBasedQueueConsumer consumer) { this.consumer = consumer; } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieProducer.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieProducer.java index d9f6001abecdb..f56dd4cce9944 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieProducer.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieProducer.java @@ -18,6 +18,11 @@ package org.apache.hudi.common.util.queue; +/** + * Producer for {@link HoodieMessageQueue}. Memory Bounded Buffer supports multiple producers single consumer pattern. + * + * @param Input type for buffer items produced + */ public interface HoodieProducer { /** diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/FormatUtils.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/FormatUtils.java index f44ec67e5e2f0..2f70a10077fc0 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/FormatUtils.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/FormatUtils.java @@ -28,7 +28,7 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.ExternalSpillableMap; import org.apache.hudi.common.util.queue.BoundedInMemoryExecutor; -import org.apache.hudi.common.util.queue.BoundedInMemoryQueueProducer; +import org.apache.hudi.common.util.queue.HoodieProducer; import org.apache.hudi.common.util.queue.FunctionBasedQueueProducer; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.configuration.FlinkOptions; @@ -229,8 +229,8 @@ public Iterator> getRecordsIterator() { /** * Setup log and parquet reading in parallel. Both write to central buffer. */ - private List>> getParallelProducers() { - List>> producers = new ArrayList<>(); + private List>> getParallelProducers() { + List>> producers = new ArrayList<>(); producers.add(new FunctionBasedQueueProducer<>(buffer -> { scanner.scan(); return null; From 3e44fb2397bdbfa9c63fab47e62f312451b91900 Mon Sep 17 00:00:00 2001 From: yuezhang Date: Thu, 6 Oct 2022 23:14:52 +0800 Subject: [PATCH 25/50] address comments --- .../org/apache/hudi/common/util/queue/HoodieMessageQueue.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieMessageQueue.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieMessageQueue.java index 0e6dde13b043a..9392136753652 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieMessageQueue.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieMessageQueue.java @@ -28,7 +28,7 @@ * 2. get record from internal message queue. * 3. close internal message queue. */ -public interface HoodieMessageQueue extends Iterable{ +public interface HoodieMessageQueue extends Iterable { /** * Get the size of inner message queue. From 672fdc6665cb250678c0768f55c8086a3cbfc388 Mon Sep 17 00:00:00 2001 From: yuezhang Date: Fri, 7 Oct 2022 11:43:27 +0800 Subject: [PATCH 26/50] address comments --- .../util/queue/BoundedInMemoryExecutor.java | 58 ++-------- .../common/util/queue/DisruptorExecutor.java | 53 +++------ .../util/queue/DisruptorMessageQueue.java | 12 +- .../common/util/queue/HoodieExecutorBase.java | 105 ++++++++++++++++++ 4 files changed, 142 insertions(+), 86 deletions(-) create mode 100644 hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieExecutorBase.java diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryExecutor.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryExecutor.java index b5fe96f9a641a..4709fe2861f4f 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryExecutor.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryExecutor.java @@ -18,7 +18,6 @@ package org.apache.hudi.common.util.queue; -import org.apache.hudi.common.util.CustomizedThreadFactory; import org.apache.hudi.common.util.DefaultSizeEstimator; import org.apache.hudi.common.util.Functions; import org.apache.hudi.common.util.Option; @@ -35,10 +34,7 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorCompletionService; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; import java.util.concurrent.Future; -import java.util.concurrent.TimeUnit; import java.util.function.Function; import java.util.stream.Collectors; @@ -50,19 +46,9 @@ public class BoundedInMemoryExecutor implements HoodieExecutor { private static final Logger LOG = LogManager.getLogger(BoundedInMemoryExecutor.class); - private static final long TERMINATE_WAITING_TIME_SECS = 60L; - // 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 BoundedInMemoryQueue 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; + private final HoodieExecutorBase hoodieExecutorBase; public BoundedInMemoryExecutor(final long bufferLimitInBytes, final Iterator inputItr, IteratorBasedQueueConsumer consumer, Function transformFunction, Runnable preExecuteRunnable) { @@ -82,14 +68,8 @@ public BoundedInMemoryExecutor(final long bufferLimitInBytes, HoodieProducer public BoundedInMemoryExecutor(final long bufferLimitInBytes, List> producers, Option> consumer, final Function transformFunction, final SizeEstimator sizeEstimator, 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 BoundedInMemoryQueue<>(bufferLimitInBytes, transformFunction, sizeEstimator); + this.hoodieExecutorBase = new HoodieExecutorBase<>(producers, consumer, preExecuteRunnable); } /** @@ -97,13 +77,13 @@ public BoundedInMemoryExecutor(final long bufferLimitInBytes, List startProducers() { // Latch to control when and which producer thread will close the queue - final CountDownLatch latch = new CountDownLatch(producers.size()); + final CountDownLatch latch = new CountDownLatch(hoodieExecutorBase.getProducers().size()); final ExecutorCompletionService completionService = - new ExecutorCompletionService(producerExecutorService); - producers.stream().map(producer -> { + new ExecutorCompletionService(hoodieExecutorBase.getProducerExecutorService()); + hoodieExecutorBase.getProducers().stream().map(producer -> { return completionService.submit(() -> { try { - preExecuteRunnable.run(); + hoodieExecutorBase.getPreExecuteRunnable().run(); producer.produce(queue); } catch (Throwable e) { LOG.error("error producing records", e); @@ -129,10 +109,10 @@ public ExecutorCompletionService startProducers() { */ @Override public Future startConsumer() { - return consumer.map(consumer -> { - return consumerExecutorService.submit(() -> { + return hoodieExecutorBase.getConsumer().map(consumer -> { + return hoodieExecutorBase.getConsumerExecutorService().submit(() -> { LOG.info("starting consumer thread"); - preExecuteRunnable.run(); + hoodieExecutorBase.getPreExecuteRunnable().run(); try { E result = consumer.consume(queue); LOG.info("Queue Consumption is done; notifying producer threads"); @@ -174,29 +154,13 @@ public E finishConsuming(Object o) throws ExecutionException, InterruptedExcepti } public void shutdownNow() { - producerExecutorService.shutdownNow(); - consumerExecutorService.shutdownNow(); + hoodieExecutorBase.shutdownNow(); // close queue to force producer stop queue.close(); } public boolean awaitTermination() { - // if current thread has been interrupted before awaitTermination was called, we still give - // executor a chance to proceeding. So clear the interrupt flag and reset it if needed before return. - boolean interruptedBefore = Thread.interrupted(); - boolean producerTerminated = false; - boolean consumerTerminated = false; - try { - producerTerminated = producerExecutorService.awaitTermination(TERMINATE_WAITING_TIME_SECS, TimeUnit.SECONDS); - consumerTerminated = consumerExecutorService.awaitTermination(TERMINATE_WAITING_TIME_SECS, TimeUnit.SECONDS); - } catch (InterruptedException ie) { - // fail silently for any other interruption - } - // reset interrupt flag if needed - if (interruptedBefore) { - Thread.currentThread().interrupt(); - } - return producerTerminated && consumerTerminated; + return hoodieExecutorBase.awaitTermination(); } public BoundedInMemoryQueue getQueue() { diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorExecutor.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorExecutor.java index 70159c0cdc506..ea69ea348ec84 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorExecutor.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorExecutor.java @@ -18,10 +18,9 @@ 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.common.util.ValidationUtils; import org.apache.hudi.exception.HoodieException; import org.apache.log4j.LogManager; @@ -32,8 +31,6 @@ 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.concurrent.Future; import java.util.function.Function; import java.util.stream.Collectors; @@ -42,18 +39,9 @@ public class DisruptorExecutor implements 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; + private final HoodieExecutorBase hoodieExecutorBase; public DisruptorExecutor(final int bufferSize, final Iterator inputItr, IteratorBasedQueueConsumer consumer, Function transformFunction, String waitStrategy, Runnable preExecuteRunnable) { @@ -73,14 +61,8 @@ public DisruptorExecutor(final int bufferSize, HoodieProducer producer, 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); + this.hoodieExecutorBase = new HoodieExecutorBase<>(producers, consumer, preExecuteRunnable); } /** @@ -88,11 +70,11 @@ public DisruptorExecutor(final int bufferSize, List> producers */ public ExecutorCompletionService startProducers() { final ExecutorCompletionService completionService = - new ExecutorCompletionService(producerExecutorService); - producers.stream().map(producer -> { + new ExecutorCompletionService(hoodieExecutorBase.getProducerExecutorService()); + hoodieExecutorBase.getProducers().stream().map(producer -> { return completionService.submit(() -> { try { - preExecuteRunnable.run(); + hoodieExecutorBase.getPreExecuteRunnable().run(); DisruptorPublisher publisher = new DisruptorPublisher<>(producer, queue); publisher.startProduce(); @@ -110,7 +92,7 @@ public ExecutorCompletionService startProducers() { @Override public E execute() { try { - assert consumer.isPresent(); + ValidationUtils.checkState(hoodieExecutorBase.getConsumer().isPresent()); startConsumer(); ExecutorCompletionService pool = startProducers(); return finishConsuming(pool); @@ -127,14 +109,14 @@ public E execute() { public E finishConsuming(Object o) throws ExecutionException, InterruptedException { ExecutorCompletionService pool = (ExecutorCompletionService) o; waitForProducersFinished(pool); - queue.getInnerQueue().shutdown(); - consumer.get().finish(); + queue.close(); + hoodieExecutorBase.getConsumer().get().finish(); - return consumer.get().getResult(); + return hoodieExecutorBase.getConsumer().get().getResult(); } private void waitForProducersFinished(ExecutorCompletionService pool) throws InterruptedException, ExecutionException { - for (int i = 0; i < producers.size(); i++) { + for (int i = 0; i < hoodieExecutorBase.getProducers().size(); i++) { pool.take().get(); } } @@ -144,11 +126,9 @@ private void waitForProducersFinished(ExecutorCompletionService pool) t */ @Override public Future startConsumer() { - DisruptorMessageHandler handler = new DisruptorMessageHandler<>(consumer.get()); - - Disruptor> innerQueue = queue.getInnerQueue(); - innerQueue.handleEventsWith(handler); - innerQueue.start(); + DisruptorMessageHandler handler = new DisruptorMessageHandler<>(hoodieExecutorBase.getConsumer().get()); + queue.setHandlers(handler); + queue.start(); return null; } @@ -159,8 +139,7 @@ public boolean isRemaining() { @Override public void shutdownNow() { - producerExecutorService.shutdownNow(); - consumerExecutorService.shutdownNow(); + hoodieExecutorBase.shutdownNow(); queue.close(); } @@ -171,6 +150,6 @@ public DisruptorMessageQueue getQueue() { @Override public boolean awaitTermination() { - return true; + return hoodieExecutorBase.awaitTermination(); } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageQueue.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageQueue.java index 9b01b24a6fb75..93043cfcef673 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageQueue.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageQueue.java @@ -33,7 +33,7 @@ public class DisruptorMessageQueue implements HoodieMessageQueue { private final Disruptor> queue; private final Function transformFunction; - private RingBuffer> ringBuffer; + private final RingBuffer> ringBuffer; public DisruptorMessageQueue(int bufferSize, Function transformFunction, String waitStrategyName, int producerNumber, Runnable preExecuteRunnable) { WaitStrategy waitStrategy = WaitStrategyFactory.build(waitStrategyName); @@ -79,7 +79,7 @@ public void close() { queue.shutdown(); } - public Disruptor> getInnerQueue() { + private Disruptor> getInnerQueue() { return this.queue; } @@ -94,4 +94,12 @@ public Iterator iterator() { dummyList.add(ringBuffer.get(0).get()); return dummyList.iterator(); } + + public void setHandlers(DisruptorMessageHandler handler) { + queue.handleEventsWith(handler); + } + + public void start() { + queue.start(); + } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieExecutorBase.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieExecutorBase.java new file mode 100644 index 0000000000000..cb9f48c2c85da --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieExecutorBase.java @@ -0,0 +1,105 @@ +/* + * 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.apache.hudi.common.util.CustomizedThreadFactory; +import org.apache.hudi.common.util.Option; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import java.util.List; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; + +/** + * HoodieExecutorBase holds common elements producerExecutorService, consumerExecutorService, producers and consumer. + * Also HoodieExecutorBase control the lifecycle of producerExecutorService and consumerExecutorService. + */ +public class HoodieExecutorBase { + + private static final Logger LOG = LogManager.getLogger(DisruptorExecutor.class); + + private static final long TERMINATE_WAITING_TIME_SECS = 60L; + // Executor service used for launching write thread. + private final ExecutorService producerExecutorService; + // Executor service used for launching read thread. + private final ExecutorService consumerExecutorService; + // 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 HoodieExecutorBase(List> producers, Option> consumer, + 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")); + } + + public Runnable getPreExecuteRunnable() { + return preExecuteRunnable; + } + + public ExecutorService getConsumerExecutorService() { + return consumerExecutorService; + } + + public ExecutorService getProducerExecutorService() { + return producerExecutorService; + } + + public List> getProducers() { + return producers; + } + + public Option> getConsumer() { + return consumer; + } + + public boolean awaitTermination() { + // if current thread has been interrupted before awaitTermination was called, we still give + // executor a chance to proceeding. So clear the interrupt flag and reset it if needed before return. + boolean interruptedBefore = Thread.interrupted(); + boolean producerTerminated = false; + boolean consumerTerminated = false; + try { + producerTerminated = producerExecutorService.awaitTermination(TERMINATE_WAITING_TIME_SECS, TimeUnit.SECONDS); + consumerTerminated = consumerExecutorService.awaitTermination(TERMINATE_WAITING_TIME_SECS, TimeUnit.SECONDS); + } catch (InterruptedException ie) { + // fail silently for any other interruption + } + // reset interrupt flag if needed + if (interruptedBefore) { + Thread.currentThread().interrupt(); + } + return producerTerminated && consumerTerminated; + } + + public void shutdownNow() { + producerExecutorService.shutdownNow(); + consumerExecutorService.shutdownNow(); + } +} From c0c22741ed123bd13cff21dec9fb7efdc70b99f8 Mon Sep 17 00:00:00 2001 From: yuezhang Date: Fri, 7 Oct 2022 12:46:11 +0800 Subject: [PATCH 27/50] address comments --- .../view/AbstractTableFileSystemView.java | 2 +- .../util/queue/BoundedInMemoryQueue.java | 2 +- .../util/queue/DisruptorMessageQueue.java | 10 ------- .../common/util/queue/HoodieMessageQueue.java | 6 +--- .../IteratorBasedHoodieMessageQueue.java | 29 +++++++++++++++++++ .../queue/IteratorBasedQueueConsumer.java | 3 +- 6 files changed, 34 insertions(+), 18 deletions(-) create mode 100644 hudi-common/src/main/java/org/apache/hudi/common/util/queue/IteratorBasedHoodieMessageQueue.java diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/AbstractTableFileSystemView.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/AbstractTableFileSystemView.java index d97127235bb89..976217ae07b78 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/view/AbstractTableFileSystemView.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/AbstractTableFileSystemView.java @@ -1,5 +1,5 @@ /* - * LicresolveCommitConflictnsed to the Apache Software Foundation (ASF) under one + * 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 diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryQueue.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryQueue.java index 9f990a3935ed4..b23db3a259593 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryQueue.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryQueue.java @@ -49,7 +49,7 @@ * @param input payload data type * @param output payload data type */ -public class BoundedInMemoryQueue implements HoodieMessageQueue { +public class BoundedInMemoryQueue extends IteratorBasedHoodieMessageQueue { /** Interval used for polling records in the queue. **/ public static final int RECORD_POLL_INTERVAL_SEC = 1; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageQueue.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageQueue.java index 93043cfcef673..b5c92546b126d 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageQueue.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageQueue.java @@ -25,8 +25,6 @@ import com.lmax.disruptor.dsl.ProducerType; import org.apache.hudi.common.util.Option; -import java.util.ArrayList; -import java.util.Iterator; import java.util.function.Function; public class DisruptorMessageQueue implements HoodieMessageQueue { @@ -87,14 +85,6 @@ public boolean isEmpty() { return ringBuffer.getBufferSize() == ringBuffer.remainingCapacity(); } - @Override - public Iterator iterator() { - // fake a dummy iterator, because disruptor engine can take care of iterating consuming records. - ArrayList dummyList = new ArrayList<>(); - dummyList.add(ringBuffer.get(0).get()); - return dummyList.iterator(); - } - public void setHandlers(DisruptorMessageHandler handler) { queue.handleEventsWith(handler); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieMessageQueue.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieMessageQueue.java index 9392136753652..6f3137026129e 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieMessageQueue.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieMessageQueue.java @@ -20,15 +20,13 @@ import org.apache.hudi.common.util.Option; -import java.util.Iterator; - /** * HoodieMessageQueue holds an internal message queue, and control the behavior of * 1. insert record into internal message queue. * 2. get record from internal message queue. * 3. close internal message queue. */ -public interface HoodieMessageQueue extends Iterable { +public interface HoodieMessageQueue { /** * Get the size of inner message queue. @@ -46,6 +44,4 @@ public interface HoodieMessageQueue extends Iterable { Option readNextRecord(); void close(); - - Iterator iterator(); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/IteratorBasedHoodieMessageQueue.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/IteratorBasedHoodieMessageQueue.java new file mode 100644 index 0000000000000..566f7bf7dea57 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/IteratorBasedHoodieMessageQueue.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; + +import java.util.Iterator; + +/** + * IteratorBasedHoodieMessageQueue implements HoodieMessageQueue with Iterable + */ +public abstract class IteratorBasedHoodieMessageQueue implements HoodieMessageQueue, Iterable{ + + public abstract Iterator iterator(); +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/IteratorBasedQueueConsumer.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/IteratorBasedQueueConsumer.java index 1b6c8a93494f6..5fde765432caf 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/IteratorBasedQueueConsumer.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/IteratorBasedQueueConsumer.java @@ -32,7 +32,8 @@ public abstract class IteratorBasedQueueConsumer implements HoodieConsumer */ @Override public O consume(HoodieMessageQueue queue) throws Exception { - Iterator iterator = queue.iterator(); + + Iterator iterator = ((IteratorBasedHoodieMessageQueue) queue).iterator(); while (iterator.hasNext()) { consumeOneRecord(iterator.next()); From 4587303118918c5e56ecb10732d9fcba43a90ee7 Mon Sep 17 00:00:00 2001 From: yuezhang Date: Fri, 7 Oct 2022 16:21:23 +0800 Subject: [PATCH 28/50] address comments --- .../hudi/common/util/queue/HoodieDisruptorEventFactory.java | 4 ++++ .../common/util/queue/IteratorBasedHoodieMessageQueue.java | 2 +- 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieDisruptorEventFactory.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieDisruptorEventFactory.java index 73ae469bc69df..e4d59f25246d6 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieDisruptorEventFactory.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieDisruptorEventFactory.java @@ -20,6 +20,10 @@ import com.lmax.disruptor.EventFactory; +/** + * Wrap a message/record as an disruptor event. + * @param + */ public class HoodieDisruptorEventFactory implements EventFactory> { @Override diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/IteratorBasedHoodieMessageQueue.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/IteratorBasedHoodieMessageQueue.java index 566f7bf7dea57..06edcd66d6da4 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/IteratorBasedHoodieMessageQueue.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/IteratorBasedHoodieMessageQueue.java @@ -23,7 +23,7 @@ /** * IteratorBasedHoodieMessageQueue implements HoodieMessageQueue with Iterable */ -public abstract class IteratorBasedHoodieMessageQueue implements HoodieMessageQueue, Iterable{ +public abstract class IteratorBasedHoodieMessageQueue implements HoodieMessageQueue, Iterable { public abstract Iterator iterator(); } From abd8b27e1c5d6761754929cdb2554b20130c9b5c Mon Sep 17 00:00:00 2001 From: yuezhang Date: Fri, 7 Oct 2022 18:20:40 +0800 Subject: [PATCH 29/50] address comments --- .../apache/hudi/config/HoodieWriteConfig.java | 10 ++-- .../execution/SparkLazyInsertIterable.java | 19 ++----- .../apache/hudi/util/MessageQueueFactory.java | 52 +++++++++++++++++++ hudi-common/pom.xml | 1 + .../util/queue/BoundedInMemoryExecutor.java | 2 +- .../common/util/queue/DisruptorExecutor.java | 2 +- .../util/queue/DisruptorMessageQueue.java | 9 +--- .../util/queue/DisruptorWaitStrategyType.java | 8 +-- .../hudi/common/util/queue/ExecutorType.java | 4 +- .../common/util/queue/HoodieConsumer.java | 3 ++ .../util/queue/HoodieDaemonThreadFactory.java | 7 ++- .../util/queue/HoodieDisruptorEvent.java | 7 +++ .../queue/HoodieDisruptorEventFactory.java | 4 +- .../common/util/queue/HoodieMessageQueue.java | 5 +- .../util/queue/WaitStrategyFactory.java | 12 ++--- .../BoundInMemoryExecutorBenchmark.scala | 2 +- pom.xml | 6 --- 17 files changed, 100 insertions(+), 53 deletions(-) create mode 100644 hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/util/MessageQueueFactory.java diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java index 0b93b06f833f9..5351cc8601df6 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java @@ -136,10 +136,10 @@ public class HoodieWriteConfig extends HoodieConfig { public static final ConfigProperty EXECUTOR_TYPE = ConfigProperty .key("hoodie.write.executor.type") - .defaultValue("BOUNDED_IN_MEMORY_EXECUTOR") + .defaultValue("BOUNDED_IN_MEMORY") .withDocumentation("Set executor which orchestrates concurrent producers and consumers communicating through a message queue." - + "default value is BOUNDED_IN_MEMORY_EXECUTOR which use a bounded in-memory queue using LinkedBlockingQueue." - + "Also users could use DISRUPTOR_EXECUTOR, which use disruptor as a lock free message queue " + + "default value is BOUNDED_IN_MEMORY which use a bounded in-memory queue using LinkedBlockingQueue." + + "Also users could use DISRUPTOR, which use disruptor as a lock free message queue " + "to gain better writing performance. Although DISRUPTOR_EXECUTOR is still an experimental feature."); public static final ConfigProperty KEYGENERATOR_TYPE = ConfigProperty @@ -250,8 +250,8 @@ public class HoodieWriteConfig extends HoodieConfig { public static final ConfigProperty WRITE_WAIT_STRATEGY = ConfigProperty .key("hoodie.write.executor.disruptor.wait.strategy") - .defaultValue("Blocking_Wait_Strategy") - .withDocumentation("Strategy employed for making DisruptorExecutor wait on a cursor."); + .defaultValue("BLOCKING_WAIT") + .withDocumentation("Strategy employed for making Disruptor Executor wait on a cursor."); public static final ConfigProperty COMBINE_BEFORE_INSERT = ConfigProperty .key("hoodie.combine.before.insert") diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/SparkLazyInsertIterable.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/SparkLazyInsertIterable.java index 9f79110ded634..eae6fb3bb3ab9 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/SparkLazyInsertIterable.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/SparkLazyInsertIterable.java @@ -26,6 +26,7 @@ import org.apache.hudi.common.util.queue.BoundedInMemoryExecutor; import org.apache.hudi.common.util.queue.DisruptorExecutor; import org.apache.hudi.common.util.queue.ExecutorType; +import org.apache.hudi.common.util.queue.HoodieConsumer; import org.apache.hudi.common.util.queue.HoodieExecutor; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieException; @@ -33,9 +34,11 @@ import org.apache.hudi.table.HoodieTable; import org.apache.avro.Schema; +import org.apache.hudi.util.MessageQueueFactory; import java.util.Iterator; import java.util.List; +import java.util.function.Function; public class SparkLazyInsertIterable extends HoodieLazyInsertIterable { @@ -87,20 +90,8 @@ protected List computeNext() { schema = HoodieAvroUtils.addMetadataFields(schema); } - ExecutorType executorType = hoodieConfig.getExecutorType(); - - switch (executorType) { - case BOUNDED_IN_MEMORY_EXECUTOR: - bufferedIteratorExecutor = new BoundedInMemoryExecutor<>(hoodieConfig.getWriteBufferLimitBytes(), inputItr, getInsertHandler(), - getTransformFunction(schema, hoodieConfig), hoodieTable.getPreExecuteRunnable()); - break; - case DISRUPTOR_EXECUTOR: - bufferedIteratorExecutor = new DisruptorExecutor<>(hoodieConfig.getWriteBufferSize(), inputItr, getInsertHandler(), - getTransformFunction(schema, hoodieConfig), hoodieConfig.getWriteWaitStrategy(), hoodieTable.getPreExecuteRunnable()); - break; - default: - throw new HoodieException("Unsupported Executor Type " + executorType); - } + bufferedIteratorExecutor = MessageQueueFactory.create(hoodieConfig, inputItr, getInsertHandler(), + getTransformFunction(schema, hoodieConfig), hoodieTable.getPreExecuteRunnable()); final List result = bufferedIteratorExecutor.execute(); assert result != null && !result.isEmpty() && !bufferedIteratorExecutor.isRemaining(); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/util/MessageQueueFactory.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/util/MessageQueueFactory.java new file mode 100644 index 0000000000000..e1fcc239ddc41 --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/util/MessageQueueFactory.java @@ -0,0 +1,52 @@ +/* + * 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.util; + +import org.apache.hudi.common.util.queue.BoundedInMemoryExecutor; +import org.apache.hudi.common.util.queue.DisruptorExecutor; +import org.apache.hudi.common.util.queue.ExecutorType; +import org.apache.hudi.common.util.queue.HoodieExecutor; +import org.apache.hudi.common.util.queue.IteratorBasedQueueConsumer; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieException; + +import java.util.Iterator; +import java.util.function.Function; + +public class MessageQueueFactory { + + /** + * Create a new hoodie executor instance on demand. + */ + public static HoodieExecutor create(HoodieWriteConfig hoodieConfig, Iterator inputItr, IteratorBasedQueueConsumer consumer, + Function transformFunction, Runnable preExecuteRunnable) { + ExecutorType executorType = hoodieConfig.getExecutorType(); + + switch (executorType) { + case BOUNDED_IN_MEMORY: + return new BoundedInMemoryExecutor<>(hoodieConfig.getWriteBufferLimitBytes(), inputItr, consumer, + transformFunction, preExecuteRunnable); + case DISRUPTOR: + return new DisruptorExecutor<>(hoodieConfig.getWriteBufferSize(), inputItr, consumer, + transformFunction, hoodieConfig.getWriteWaitStrategy(), preExecuteRunnable); + default: + throw new HoodieException("Unsupported Executor Type " + executorType); + } + } +} diff --git a/hudi-common/pom.xml b/hudi-common/pom.xml index 938222b7f4742..23021a3273c0c 100644 --- a/hudi-common/pom.xml +++ b/hudi-common/pom.xml @@ -302,6 +302,7 @@ com.lmax disruptor + ${disruptor.version} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryExecutor.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryExecutor.java index 4709fe2861f4f..747018123ecce 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryExecutor.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryExecutor.java @@ -88,7 +88,7 @@ public ExecutorCompletionService startProducers() { } catch (Throwable e) { LOG.error("error producing records", e); queue.markAsFailed(e); - throw e; + throw new HoodieException("Error producing records in bounded in memory executor", e); } finally { synchronized (latch) { latch.countDown(); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorExecutor.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorExecutor.java index ea69ea348ec84..21b88e30db140 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorExecutor.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorExecutor.java @@ -81,7 +81,7 @@ public ExecutorCompletionService startProducers() { } catch (Throwable e) { LOG.error("error producing records", e); - throw e; + throw new HoodieException("Error producing records in disruptor executor", e); } return true; }); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageQueue.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageQueue.java index b5c92546b126d..14297e41c9f44 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageQueue.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageQueue.java @@ -33,16 +33,11 @@ public class DisruptorMessageQueue implements HoodieMessageQueue { private final Function transformFunction; private final RingBuffer> ringBuffer; - public DisruptorMessageQueue(int bufferSize, Function transformFunction, String waitStrategyName, int producerNumber, Runnable preExecuteRunnable) { + public DisruptorMessageQueue(int bufferSize, Function transformFunction, String waitStrategyName, int totalProducers, 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.queue = new Disruptor<>(HoodieDisruptorEvent::new, bufferSize, threadFactory, totalProducers > 1 ? ProducerType.MULTI : ProducerType.SINGLE, waitStrategy); this.ringBuffer = queue.getRingBuffer(); this.transformFunction = transformFunction; } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorWaitStrategyType.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorWaitStrategyType.java index 45c086cda9ad3..1a8e86835d0cf 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorWaitStrategyType.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorWaitStrategyType.java @@ -30,14 +30,14 @@ public enum DisruptorWaitStrategyType { * The BlockingWaitStrategy is the slowest of the available wait strategies, but is the most conservative with the respect to CPU usage * and will give the most consistent behaviour across the widest variety of deployment options. */ - BLOCKING_WAIT_STRATEGY, + BLOCKING_WAIT, /** * Like the `BlockingWaitStrategy` the `SleepingWaitStrategy` it attempts to be conservative with CPU usage by using a simple busy wait loop. * The difference is that the `SleepingWaitStrategy` uses a call to `LockSupport.parkNanos(1)` in the middle of the loop. * On a typical Linux system this will pause the thread for around 60µs. */ - SLEEPING_WAIT_STRATEGY, + SLEEPING_WAIT, /** * The `YieldingWaitStrategy` is one of two WaitStrategies that can be use in low-latency systems. @@ -47,13 +47,13 @@ public enum DisruptorWaitStrategyType { * This is the recommended wait strategy when you need very high performance, and the number of `EventHandler` threads is lower than the total number of logical cores, * e.g. you have hyper-threading enabled. */ - YIELDING_WAIT_SRATEGY, + YIELDING_WAIT, /** * The `BusySpinWaitStrategy` is the highest performing WaitStrategy. * Like the `YieldingWaitStrategy`, it can be used in low-latency systems, but puts the highest constraints on the deployment environment. */ - BUSY_SPIN_WAIT_STRATEGY; + BUSY_SPIN_WAIT; public static List getNames() { List names = new ArrayList<>(KeyGeneratorType.values().length); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/ExecutorType.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/ExecutorType.java index 959f9979e0940..08d37ad1d7ecf 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/ExecutorType.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/ExecutorType.java @@ -32,13 +32,13 @@ public enum ExecutorType { /** * Executor which orchestrates concurrent producers and consumers communicating through a bounded in-memory message queue using LinkedBlockingQueue. */ - BOUNDED_IN_MEMORY_EXECUTOR, + BOUNDED_IN_MEMORY, /** * Executor which orchestrates concurrent producers and consumers communicating through disruptor as a lock free message queue * to gain better writing performance. Although DisruptorExecutor is still an experimental feature. */ - DISRUPTOR_EXECUTOR; + DISRUPTOR; public static List getNames() { List names = new ArrayList<>(KeyGeneratorType.values().length); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieConsumer.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieConsumer.java index 0c5f484fcd68b..fb67fab6c74c7 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieConsumer.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieConsumer.java @@ -18,6 +18,9 @@ package org.apache.hudi.common.util.queue; +/** + * HoodieConsumer is used to consume records/messages from hoodie inner message queue and write into DFS. + */ public interface HoodieConsumer { /** diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieDaemonThreadFactory.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieDaemonThreadFactory.java index 77f0eb6a0d888..243f3a81e57b3 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieDaemonThreadFactory.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieDaemonThreadFactory.java @@ -21,9 +21,14 @@ import java.util.concurrent.ThreadFactory; import java.util.concurrent.atomic.AtomicInteger; +/** + * Access to a ThreadFactory instance. + * 1. All threads are created with setDaemon(true). + * 2. All threads execute preExecuteRunnable func once. + */ public class HoodieDaemonThreadFactory implements ThreadFactory { - private Runnable preExecuteRunnable; + private final Runnable preExecuteRunnable; private final AtomicInteger threadsNum = new AtomicInteger(); private final String namePattern; private final String baseName = "Hoodie-disruptor-daemon-thread"; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieDisruptorEvent.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieDisruptorEvent.java index 4f64132bef765..9fd4e13addb16 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieDisruptorEvent.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieDisruptorEvent.java @@ -18,6 +18,9 @@ package org.apache.hudi.common.util.queue; +/** + * The unit of data passed from producer to consumer in disruptor world. + */ public class HoodieDisruptorEvent { private O value; @@ -30,6 +33,10 @@ public O get() { return this.value; } + /** + * When passing data via the Disruptor, it is possible for objects to live longer than intended. + * To avoid this happening it is necessary to clear out the event after processing it. + */ public void clear() { value = null; } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieDisruptorEventFactory.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieDisruptorEventFactory.java index e4d59f25246d6..a9aa2a558c71e 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieDisruptorEventFactory.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieDisruptorEventFactory.java @@ -21,8 +21,8 @@ import com.lmax.disruptor.EventFactory; /** - * Wrap a message/record as an disruptor event. - * @param + * HoodieDisruptorEventFactory is used to create/preallocate HoodieDisruptorEvent. + * */ public class HoodieDisruptorEventFactory implements EventFactory> { diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieMessageQueue.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieMessageQueue.java index 6f3137026129e..ce5535450e49a 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieMessageQueue.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieMessageQueue.java @@ -19,6 +19,7 @@ package org.apache.hudi.common.util.queue; import org.apache.hudi.common.util.Option; +import java.io.Closeable; /** * HoodieMessageQueue holds an internal message queue, and control the behavior of @@ -26,7 +27,7 @@ * 2. get record from internal message queue. * 3. close internal message queue. */ -public interface HoodieMessageQueue { +public interface HoodieMessageQueue extends Closeable { /** * Get the size of inner message queue. @@ -42,6 +43,4 @@ public interface HoodieMessageQueue { * Read records from inner message queue. */ Option readNextRecord(); - - void close(); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/WaitStrategyFactory.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/WaitStrategyFactory.java index c69dc3859dc0a..a749cae886272 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/WaitStrategyFactory.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/WaitStrategyFactory.java @@ -18,7 +18,7 @@ package org.apache.hudi.common.util.queue; -import static org.apache.hudi.common.util.queue.DisruptorWaitStrategyType.BLOCKING_WAIT_STRATEGY; +import static org.apache.hudi.common.util.queue.DisruptorWaitStrategyType.BLOCKING_WAIT; import com.lmax.disruptor.BlockingWaitStrategy; import com.lmax.disruptor.BusySpinWaitStrategy; @@ -29,7 +29,7 @@ public class WaitStrategyFactory { - public static final String DEFAULT_STRATEGY = BLOCKING_WAIT_STRATEGY.name(); + public static final String DEFAULT_STRATEGY = BLOCKING_WAIT.name(); /** * Build WaitStrategy for disruptor @@ -38,13 +38,13 @@ public static WaitStrategy build(String name) { DisruptorWaitStrategyType strategyType = DisruptorWaitStrategyType.valueOf(name.toUpperCase()); switch (strategyType) { - case BLOCKING_WAIT_STRATEGY: + case BLOCKING_WAIT: return new BlockingWaitStrategy(); - case SLEEPING_WAIT_STRATEGY: + case SLEEPING_WAIT: return new SleepingWaitStrategy(); - case YIELDING_WAIT_SRATEGY: + case YIELDING_WAIT: return new YieldingWaitStrategy(); - case BUSY_SPIN_WAIT_STRATEGY: + case BUSY_SPIN_WAIT: return new BusySpinWaitStrategy(); default: throw new HoodieException("Unsupported Executor Type " + name); diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/execution/benchmark/BoundInMemoryExecutorBenchmark.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/execution/benchmark/BoundInMemoryExecutorBenchmark.scala index be9110e1dcf00..b1d2517374c7e 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/execution/benchmark/BoundInMemoryExecutorBenchmark.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/execution/benchmark/BoundInMemoryExecutorBenchmark.scala @@ -115,7 +115,7 @@ object BoundInMemoryExecutorBenchmark extends HoodieBenchmarkBase { .option("hoodie.bulkinsert.shuffle.parallelism", "1") .option("hoodie.upsert.shuffle.parallelism", "2") .option("hoodie.delete.shuffle.parallelism", "2") - .option("hoodie.write.executor.type", "DISRUPTOR_EXECUTOR") + .option("hoodie.write.executor.type", "DISRUPTOR") .option("hoodie.populate.meta.fields", "false") .option("hoodie.table.keygenerator.class", "org.apache.hudi.keygen.SimpleKeyGenerator") diff --git a/pom.xml b/pom.xml index 3437cd2631406..7b915d351e7c7 100644 --- a/pom.xml +++ b/pom.xml @@ -1405,12 +1405,6 @@ ${zk-curator.version} - - com.lmax - disruptor - ${disruptor.version} - - com.google.protobuf From 4ba91d4ce8345b4917e1f402694a55d07bf2951c Mon Sep 17 00:00:00 2001 From: yuezhang Date: Fri, 7 Oct 2022 18:59:02 +0800 Subject: [PATCH 30/50] address comments --- .../org/apache/hudi/execution/SparkLazyInsertIterable.java | 5 ----- 1 file changed, 5 deletions(-) diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/SparkLazyInsertIterable.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/SparkLazyInsertIterable.java index eae6fb3bb3ab9..1e196f3ce0f5e 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/SparkLazyInsertIterable.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/SparkLazyInsertIterable.java @@ -23,10 +23,6 @@ import org.apache.hudi.common.engine.TaskContextSupplier; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; -import org.apache.hudi.common.util.queue.BoundedInMemoryExecutor; -import org.apache.hudi.common.util.queue.DisruptorExecutor; -import org.apache.hudi.common.util.queue.ExecutorType; -import org.apache.hudi.common.util.queue.HoodieConsumer; import org.apache.hudi.common.util.queue.HoodieExecutor; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieException; @@ -38,7 +34,6 @@ import java.util.Iterator; import java.util.List; -import java.util.function.Function; public class SparkLazyInsertIterable extends HoodieLazyInsertIterable { From 70ca2c723ddac5acc8f99e1e45968fc4fe61937c Mon Sep 17 00:00:00 2001 From: yuezhang Date: Sat, 8 Oct 2022 18:33:02 +0800 Subject: [PATCH 31/50] add mmore tests --- .../execution/TestDisruptorMessageQueue.java | 210 ++++++++++++++++++ .../common/util/queue/DisruptorExecutor.java | 1 + .../util/queue/DisruptorMessageQueue.java | 11 +- 3 files changed, 220 insertions(+), 2 deletions(-) diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorMessageQueue.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorMessageQueue.java index 3487cd8317394..3163e32905873 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorMessageQueue.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorMessageQueue.java @@ -26,10 +26,17 @@ import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.queue.DisruptorMessageHandler; +import org.apache.hudi.common.util.queue.DisruptorMessageQueue; +import org.apache.hudi.common.util.queue.DisruptorPublisher; +import org.apache.hudi.common.util.queue.FunctionBasedQueueProducer; +import org.apache.hudi.common.util.queue.HoodieProducer; import org.apache.hudi.common.util.queue.IteratorBasedQueueConsumer; import org.apache.hudi.common.util.queue.DisruptorExecutor; +import org.apache.hudi.common.util.queue.IteratorBasedQueueProducer; import org.apache.hudi.common.util.queue.WaitStrategyFactory; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieException; import org.apache.hudi.testutils.HoodieClientTestHarness; import org.apache.spark.TaskContext; import org.apache.spark.TaskContext$; @@ -40,12 +47,20 @@ import java.io.IOException; import java.util.ArrayList; +import java.util.HashMap; +import java.util.Iterator; import java.util.List; +import java.util.Map; +import java.util.concurrent.Future; +import java.util.function.Function; +import java.util.stream.Collectors; +import java.util.stream.IntStream; import scala.Tuple2; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertThrows; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -144,4 +159,199 @@ protected Integer getResult() { } } } + + /** + * Test to ensure that we are reading all records from queue iterator when we have multiple producers. + */ + @SuppressWarnings("unchecked") + @Test + @Timeout(value = 60) + public void testCompositeProducerRecordReading() throws Exception { + final int numRecords = 1000; + final int numProducers = 40; + final List> recs = new ArrayList<>(); + + final DisruptorMessageQueue queue = + new DisruptorMessageQueue(1024, getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA), + "BLOCKING_WAIT", numProducers, new Runnable() { + @Override + public void run() { + // do nothing. + } + }); + + // Record Key to + Map> keyToProducerAndIndexMap = new HashMap<>(); + + for (int i = 0; i < numProducers; i++) { + List pRecs = dataGen.generateInserts(instantTime, numRecords); + int j = 0; + for (HoodieRecord r : pRecs) { + assertFalse(keyToProducerAndIndexMap.containsKey(r.getRecordKey())); + keyToProducerAndIndexMap.put(r.getRecordKey(), new Tuple2<>(i, j)); + j++; + } + recs.add(pRecs); + } + + List disruptorPublishers = new ArrayList<>(); + for (int i = 0; i < recs.size(); i++) { + final List r = recs.get(i); + // Alternate between pull and push based iterators + if (i % 2 == 0) { + DisruptorPublisher publisher = new DisruptorPublisher<>(new IteratorBasedQueueProducer<>(r.iterator()), queue); + disruptorPublishers.add(publisher); + } else { + DisruptorPublisher publisher = new DisruptorPublisher<>(new FunctionBasedQueueProducer<>((buf) -> { + Iterator itr = r.iterator(); + while (itr.hasNext()) { + try { + buf.insertRecord(itr.next()); + } catch (Exception e) { + throw new HoodieException(e); + } + } + return true; + }), queue); + disruptorPublishers.add(publisher); + } + } + + // Used to ensure that consumer sees the records generated by a single producer in FIFO order + Map lastSeenMap = + IntStream.range(0, numProducers).boxed().collect(Collectors.toMap(Function.identity(), x -> -1)); + Map countMap = + IntStream.range(0, numProducers).boxed().collect(Collectors.toMap(Function.identity(), x -> 0)); + + + // setup consumer and start disruptor + DisruptorMessageHandler, Integer> handler = + new DisruptorMessageHandler<>(new IteratorBasedQueueConsumer, Integer>() { + + @Override + public void consumeOneRecord(HoodieLazyInsertIterable.HoodieInsertValueGenResult payload) { + // Read recs and ensure we have covered all producer recs. + final HoodieRecord rec = payload.record; + Tuple2 producerPos = keyToProducerAndIndexMap.get(rec.getRecordKey()); + Integer lastSeenPos = lastSeenMap.get(producerPos._1()); + countMap.put(producerPos._1(), countMap.get(producerPos._1()) + 1); + lastSeenMap.put(producerPos._1(), lastSeenPos + 1); + // Ensure we are seeing the next record generated + assertEquals(lastSeenPos + 1, producerPos._2().intValue()); + } + + @Override + public void finish() { + } + + @Override + protected Integer getResult() { + return 0; + } + }); + + + queue.setHandlers(handler); + queue.start(); + + + // start to produce records + final List> futureList = disruptorPublishers.stream().map(disruptorPublisher -> { + return executorService.submit(() -> { + disruptorPublisher.startProduce(); + return true; + }); + }).collect(Collectors.toList()); + + // Close queue + Future closeFuture = executorService.submit(() -> { + try { + for (Future f : futureList) { + f.get(); + } + } catch (Exception e) { + throw new RuntimeException(e); + } + return true; + }); + + // wait for producer finished + closeFuture.get(); + + // wait for all the records consumed. + queue.close(); + queue.waitForConsumingFinished(); + + for (int i = 0; i < numProducers; i++) { + // Ensure we have seen all the records for each producers + assertEquals(Integer.valueOf(numRecords), countMap.get(i)); + } + } + + /** + * Test to ensure that one of the producers exception will stop current ingestion. + */ + @SuppressWarnings("unchecked") + @Test + @Timeout(value = 60) + public void testException() throws Exception { + final int numRecords = 1000; + final int numProducers = 40; + + final DisruptorMessageQueue queue = + new DisruptorMessageQueue(1024, getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA), + "BLOCKING_WAIT", numProducers, new Runnable() { + @Override + public void run() { + // do nothing. + } + }); + + List pRecs = dataGen.generateInserts(instantTime, numRecords); + + // create 2 producers + // producer1 : common producer + // producer2 : exception producer + List producers = new ArrayList<>(); + for (int i = 0; i < 2; i++) { + if (i % 2 == 0) { + producers.add(new IteratorBasedQueueProducer<>(pRecs.iterator())); + } else { + producers.add(new FunctionBasedQueueProducer<>((buf) -> { + throw new HoodieException("Exception when produce records!!!"); + })); + } + } + + + IteratorBasedQueueConsumer, Integer> consumer = + new IteratorBasedQueueConsumer, Integer>() { + + int count = 0; + @Override + public void consumeOneRecord(HoodieLazyInsertIterable.HoodieInsertValueGenResult payload) { + // Read recs and ensure we have covered all producer recs. + final HoodieRecord rec = payload.record; + count++; + } + + @Override + public void finish() { + } + + @Override + protected Integer getResult() { + return count; + } + }; + + DisruptorExecutor>, Integer> exec = new DisruptorExecutor(1024, + producers, Option.of(consumer), getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA), + WaitStrategyFactory.DEFAULT_STRATEGY, getPreExecuteRunnable()); + + final Throwable thrown = assertThrows(HoodieException.class, exec::execute, + "exception is expected"); + assertEquals("java.util.concurrent.ExecutionException: org.apache.hudi.exception.HoodieException: Error producing records in disruptor executor", + thrown.getMessage()); + } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorExecutor.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorExecutor.java index 21b88e30db140..9769492e4b075 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorExecutor.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorExecutor.java @@ -110,6 +110,7 @@ public E finishConsuming(Object o) throws ExecutionException, InterruptedExcepti ExecutorCompletionService pool = (ExecutorCompletionService) o; waitForProducersFinished(pool); queue.close(); + queue.waitForConsumingFinished(); hoodieExecutorBase.getConsumer().get().finish(); return hoodieExecutorBase.getConsumer().get().getResult(); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageQueue.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageQueue.java index 14297e41c9f44..4845175cdcdc6 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageQueue.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageQueue.java @@ -24,11 +24,15 @@ import com.lmax.disruptor.dsl.Disruptor; import com.lmax.disruptor.dsl.ProducerType; import org.apache.hudi.common.util.Option; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; import java.util.function.Function; public class DisruptorMessageQueue implements HoodieMessageQueue { + private static final Logger LOG = LogManager.getLogger(DisruptorMessageQueue.class); + private final Disruptor> queue; private final Function transformFunction; private final RingBuffer> ringBuffer; @@ -72,8 +76,11 @@ public void close() { queue.shutdown(); } - private Disruptor> getInnerQueue() { - return this.queue; + public void waitForConsumingFinished() throws InterruptedException { + while (!isEmpty()) { + Thread.sleep(1000); + LOG.info("Still waiting for consuming finished."); + } } public boolean isEmpty() { From 447cb4510301af1c3ff1aebb3bd0a668872fc3f6 Mon Sep 17 00:00:00 2001 From: yuezhang Date: Sat, 8 Oct 2022 18:42:17 +0800 Subject: [PATCH 32/50] address comments --- .../hudi/execution/TestBoundedInMemoryExecutorInSpark.java | 1 + 1 file changed, 1 insertion(+) diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestBoundedInMemoryExecutorInSpark.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestBoundedInMemoryExecutorInSpark.java index 69457428d107e..30ebe46d6770c 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestBoundedInMemoryExecutorInSpark.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestBoundedInMemoryExecutorInSpark.java @@ -68,6 +68,7 @@ private Runnable getPreExecuteRunnable() { return () -> TaskContext$.MODULE$.setTaskContext(taskContext); } + // common produce and consume based on disruptor executor @Test public void testExecutor() { From eaf0ae0f52195649eed2ffc0f67713bf91b9d477 Mon Sep 17 00:00:00 2001 From: yuezhang Date: Fri, 14 Oct 2022 17:02:11 +0800 Subject: [PATCH 33/50] address comments --- .../apache/hudi/config/HoodieWriteConfig.java | 10 +- .../execution/SparkLazyInsertIterable.java | 4 +- ...ry.java => QueueBasedExecutorFactory.java} | 4 +- .../TestBoundedInMemoryExecutorInSpark.java | 18 +-- .../TestDisruptorExecutionInSpark.java | 70 +++++------ .../execution/TestDisruptorMessageQueue.java | 61 ++++------ .../util/queue/BoundedInMemoryExecutor.java | 52 +++------ .../util/queue/BoundedInMemoryQueue.java | 5 + .../common/util/queue/DisruptorExecutor.java | 110 ++++++++---------- .../util/queue/DisruptorMessageQueue.java | 10 +- .../util/queue/HoodieDaemonThreadFactory.java | 20 +++- .../common/util/queue/HoodieExecutor.java | 14 ++- .../common/util/queue/HoodieExecutorBase.java | 7 +- .../common/util/queue/HoodieMessageQueue.java | 7 ++ .../queue/IteratorBasedQueueConsumer.java | 2 +- .../MessageQueueBasedHoodieExecutor.java | 69 +++++++++++ .../util/queue/WaitStrategyFactory.java | 8 +- 17 files changed, 262 insertions(+), 209 deletions(-) rename hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/util/{MessageQueueFactory.java => QueueBasedExecutorFactory.java} (93%) create mode 100644 hudi-common/src/main/java/org/apache/hudi/common/util/queue/MessageQueueBasedHoodieExecutor.java diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java index d3788aa371525..04788bed752ab 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java @@ -1064,12 +1064,12 @@ public int getWriteBufferLimitBytes() { return Integer.parseInt(getStringOrDefault(WRITE_BUFFER_LIMIT_BYTES_VALUE)); } - public String getWriteWaitStrategy() { - return getString(WRITE_WAIT_STRATEGY); + public Option getWriteExecutorWaitStrategy() { + return Option.of(getString(WRITE_WAIT_STRATEGY)); } - public int getWriteBufferSize() { - return getInt(WRITE_BUFFER_SIZE); + public Option getWriteBufferSize() { + return Option.of(getInt(WRITE_BUFFER_SIZE)); } public boolean shouldCombineBeforeInsert() { @@ -2319,7 +2319,7 @@ public Builder withKeyGenerator(String keyGeneratorClass) { return this; } - public Builder withExecutorName(String executorClass) { + public Builder withExecutorType(String executorClass) { writeConfig.setValue(EXECUTOR_TYPE, executorClass); return this; } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/SparkLazyInsertIterable.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/SparkLazyInsertIterable.java index 1e196f3ce0f5e..d2555f95980b7 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/SparkLazyInsertIterable.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/SparkLazyInsertIterable.java @@ -30,7 +30,7 @@ import org.apache.hudi.table.HoodieTable; import org.apache.avro.Schema; -import org.apache.hudi.util.MessageQueueFactory; +import org.apache.hudi.util.QueueBasedExecutorFactory; import java.util.Iterator; import java.util.List; @@ -85,7 +85,7 @@ protected List computeNext() { schema = HoodieAvroUtils.addMetadataFields(schema); } - bufferedIteratorExecutor = MessageQueueFactory.create(hoodieConfig, inputItr, getInsertHandler(), + bufferedIteratorExecutor = QueueBasedExecutorFactory.create(hoodieConfig, inputItr, getInsertHandler(), getTransformFunction(schema, hoodieConfig), hoodieTable.getPreExecuteRunnable()); final List result = bufferedIteratorExecutor.execute(); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/util/MessageQueueFactory.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/util/QueueBasedExecutorFactory.java similarity index 93% rename from hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/util/MessageQueueFactory.java rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/util/QueueBasedExecutorFactory.java index e1fcc239ddc41..e5e8be9d14ea9 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/util/MessageQueueFactory.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/util/QueueBasedExecutorFactory.java @@ -29,7 +29,7 @@ import java.util.Iterator; import java.util.function.Function; -public class MessageQueueFactory { +public class QueueBasedExecutorFactory { /** * Create a new hoodie executor instance on demand. @@ -44,7 +44,7 @@ public static HoodieExecutor create(HoodieWriteConfig hoodieConfig, Iterator inp transformFunction, preExecuteRunnable); case DISRUPTOR: return new DisruptorExecutor<>(hoodieConfig.getWriteBufferSize(), inputItr, consumer, - transformFunction, hoodieConfig.getWriteWaitStrategy(), preExecuteRunnable); + transformFunction, hoodieConfig.getWriteExecutorWaitStrategy(), preExecuteRunnable); default: throw new HoodieException("Unsupported Executor Type " + executorType); } diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestBoundedInMemoryExecutorInSpark.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestBoundedInMemoryExecutorInSpark.java index 30ebe46d6770c..040634da4c335 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestBoundedInMemoryExecutorInSpark.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestBoundedInMemoryExecutorInSpark.java @@ -68,11 +68,11 @@ private Runnable getPreExecuteRunnable() { return () -> TaskContext$.MODULE$.setTaskContext(taskContext); } - // common produce and consume based on disruptor executor @Test public void testExecutor() { - final List hoodieRecords = dataGen.generateInserts(instantTime, 100); + final int recordNumber = 100; + final List hoodieRecords = dataGen.generateInserts(instantTime, recordNumber); HoodieWriteConfig hoodieWriteConfig = mock(HoodieWriteConfig.class); when(hoodieWriteConfig.getWriteBufferLimitBytes()).thenReturn(1024); @@ -86,10 +86,6 @@ public void consumeOneRecord(HoodieLazyInsertIterable.HoodieInsertValueGenResult count++; } - @Override - public void finish() { - } - @Override protected Integer getResult() { return count; @@ -101,7 +97,7 @@ protected Integer getResult() { executor = new BoundedInMemoryExecutor(hoodieWriteConfig.getWriteBufferLimitBytes(), hoodieRecords.iterator(), consumer, getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA), getPreExecuteRunnable()); int result = executor.execute(); - // It should buffer and write 100 records + assertEquals(100, result); // There should be no remaining records in the buffer assertFalse(executor.isRemaining()); @@ -133,10 +129,6 @@ public void consumeOneRecord(HoodieLazyInsertIterable.HoodieInsertValueGenResult } } - @Override - public void finish() { - } - @Override protected Integer getResult() { return 0; @@ -183,10 +175,6 @@ public GenericRecord next() { public void consumeOneRecord(HoodieLazyInsertIterable.HoodieInsertValueGenResult record) { } - @Override - public void finish() { - } - @Override protected Integer getResult() { return 0; diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorExecutionInSpark.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorExecutionInSpark.java index be057e64056ca..1e075fad22126 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorExecutionInSpark.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorExecutionInSpark.java @@ -37,16 +37,17 @@ import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import java.util.ArrayList; import java.util.List; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.Future; -import java.util.concurrent.atomic.AtomicReference; +import java.util.concurrent.locks.Condition; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; import scala.Tuple2; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -75,9 +76,10 @@ private Runnable getPreExecuteRunnable() { public void testExecutor() { final List hoodieRecords = dataGen.generateInserts(instantTime, 128); + final List consumedRecords = new ArrayList<>(); HoodieWriteConfig hoodieWriteConfig = mock(HoodieWriteConfig.class); - when(hoodieWriteConfig.getWriteBufferSize()).thenReturn(8); + when(hoodieWriteConfig.getWriteBufferSize()).thenReturn(Option.of(8)); IteratorBasedQueueConsumer, Integer> consumer = new IteratorBasedQueueConsumer, Integer>() { @@ -85,13 +87,10 @@ public void testExecutor() { @Override public void consumeOneRecord(HoodieLazyInsertIterable.HoodieInsertValueGenResult record) { + consumedRecords.add(record.record); count++; } - @Override - public void finish() { - } - @Override protected Integer getResult() { return count; @@ -101,12 +100,20 @@ protected Integer getResult() { try { exec = new DisruptorExecutor(hoodieWriteConfig.getWriteBufferSize(), hoodieRecords.iterator(), consumer, - getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA), WaitStrategyFactory.DEFAULT_STRATEGY, getPreExecuteRunnable()); + getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA), Option.of(WaitStrategyFactory.DEFAULT_STRATEGY), getPreExecuteRunnable()); int result = exec.execute(); // It should buffer and write 100 records assertEquals(128, result); // There should be no remaining records in the buffer assertFalse(exec.isRemaining()); + + // collect all records and assert that consumed records are identical to produced ones + // assert there's no tampering, and that the ordering is preserved + assertEquals(hoodieRecords.size(), consumedRecords.size()); + for (int i = 0; i < hoodieRecords.size(); i++) { + assertEquals(hoodieRecords.get(i), consumedRecords.get(i)); + } + } finally { if (exec != null) { exec.shutdownNow(); @@ -117,58 +124,41 @@ protected Integer getResult() { @Test public void testInterruptExecutor() { final List hoodieRecords = dataGen.generateInserts(instantTime, 100); - ExecutorService pool = Executors.newSingleThreadExecutor(); + final Lock lock = new ReentrantLock(); + Condition condition = lock.newCondition(); HoodieWriteConfig hoodieWriteConfig = mock(HoodieWriteConfig.class); - when(hoodieWriteConfig.getWriteBufferSize()).thenReturn(1024); + when(hoodieWriteConfig.getWriteBufferSize()).thenReturn(Option.of(1024)); IteratorBasedQueueConsumer, Integer> consumer = new IteratorBasedQueueConsumer, Integer>() { @Override public void consumeOneRecord(HoodieLazyInsertIterable.HoodieInsertValueGenResult record) { try { - while (true) { - Thread.sleep(1000); - } + condition.wait(); } catch (InterruptedException ie) { - return; + // ignore here } } - @Override - public void finish() { - } - @Override protected Integer getResult() { return 0; } }; - DisruptorExecutor>, Integer> executor = null; - AtomicReference actualException = new AtomicReference<>(); + DisruptorExecutor>, Integer> + executor = new DisruptorExecutor(hoodieWriteConfig.getWriteBufferSize(), hoodieRecords.iterator(), consumer, + getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA), Option.of(WaitStrategyFactory.DEFAULT_STRATEGY), getPreExecuteRunnable()); + try { - executor = new DisruptorExecutor(hoodieWriteConfig.getWriteBufferSize(), hoodieRecords.iterator(), consumer, - getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA), WaitStrategyFactory.DEFAULT_STRATEGY, getPreExecuteRunnable()); - DisruptorExecutor>, Integer> finalExecutor = executor; - - Future future = pool.submit(() -> { - try { - finalExecutor.execute(); - } catch (Exception e) { - actualException.set(e); - } - - }); - future.cancel(true); - future.get(); - assertTrue(actualException.get() instanceof HoodieException); + Thread.currentThread().interrupt(); + assertThrows(HoodieException.class, executor::execute); + assertTrue(Thread.interrupted()); } catch (Exception e) { // ignore here } finally { - if (executor != null) { - executor.shutdownNow(); - } + executor.shutdownNow(); } } } diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorMessageQueue.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorMessageQueue.java index 3163e32905873..efa876e8bb1af 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorMessageQueue.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorMessageQueue.java @@ -26,6 +26,7 @@ import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.common.util.queue.DisruptorMessageHandler; import org.apache.hudi.common.util.queue.DisruptorMessageQueue; import org.apache.hudi.common.util.queue.DisruptorPublisher; @@ -105,12 +106,12 @@ public void testRecordReading() throws Exception { originalRecord.getData().getInsertValue(HoodieTestDataGenerator.AVRO_SCHEMA); beforeIndexedRecord.add(originalInsertValue.get()); } catch (IOException e) { - e.printStackTrace(); + // ignore exception here. } }); HoodieWriteConfig hoodieWriteConfig = mock(HoodieWriteConfig.class); - when(hoodieWriteConfig.getWriteBufferSize()).thenReturn(16); + when(hoodieWriteConfig.getWriteBufferSize()).thenReturn(Option.of(16)); IteratorBasedQueueConsumer, Integer> consumer = new IteratorBasedQueueConsumer, Integer>() { @@ -125,14 +126,10 @@ public void consumeOneRecord(HoodieLazyInsertIterable.HoodieInsertValueGenResult .getData().getInsertValue(HoodieTestDataGenerator.AVRO_SCHEMA).get(); afterIndexedRecord.add(indexedRecord); } catch (IOException e) { - e.printStackTrace(); + //ignore exception here. } } - @Override - public void finish() { - } - @Override protected Integer getResult() { return count; @@ -143,7 +140,7 @@ protected Integer getResult() { try { exec = new DisruptorExecutor(hoodieWriteConfig.getWriteBufferSize(), hoodieRecords.iterator(), consumer, - getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA), WaitStrategyFactory.DEFAULT_STRATEGY, getPreExecuteRunnable()); + getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA), Option.of(WaitStrategyFactory.DEFAULT_STRATEGY), getPreExecuteRunnable()); int result = exec.execute(); // It should buffer and write 100 records assertEquals(100, result); @@ -172,8 +169,8 @@ public void testCompositeProducerRecordReading() throws Exception { final List> recs = new ArrayList<>(); final DisruptorMessageQueue queue = - new DisruptorMessageQueue(1024, getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA), - "BLOCKING_WAIT", numProducers, new Runnable() { + new DisruptorMessageQueue(Option.of(1024), getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA), + Option.of("BLOCKING_WAIT"), numProducers, new Runnable() { @Override public void run() { // do nothing. @@ -181,14 +178,14 @@ public void run() { }); // Record Key to - Map> keyToProducerAndIndexMap = new HashMap<>(); + Map> keyToProducerAndIndexMap = new HashMap<>(); for (int i = 0; i < numProducers; i++) { List pRecs = dataGen.generateInserts(instantTime, numRecords); int j = 0; for (HoodieRecord r : pRecs) { assertFalse(keyToProducerAndIndexMap.containsKey(r.getRecordKey())); - keyToProducerAndIndexMap.put(r.getRecordKey(), new Tuple2<>(i, j)); + keyToProducerAndIndexMap.put(r.getRecordKey(), Pair.of(i, j)); j++; } recs.add(pRecs); @@ -232,16 +229,12 @@ public void run() { public void consumeOneRecord(HoodieLazyInsertIterable.HoodieInsertValueGenResult payload) { // Read recs and ensure we have covered all producer recs. final HoodieRecord rec = payload.record; - Tuple2 producerPos = keyToProducerAndIndexMap.get(rec.getRecordKey()); - Integer lastSeenPos = lastSeenMap.get(producerPos._1()); - countMap.put(producerPos._1(), countMap.get(producerPos._1()) + 1); - lastSeenMap.put(producerPos._1(), lastSeenPos + 1); + Pair producerPos = keyToProducerAndIndexMap.get(rec.getRecordKey()); + Integer lastSeenPos = lastSeenMap.get(producerPos.getLeft()); + countMap.put(producerPos.getLeft(), countMap.get(producerPos.getLeft()) + 1); + lastSeenMap.put(producerPos.getLeft(), lastSeenPos + 1); // Ensure we are seeing the next record generated - assertEquals(lastSeenPos + 1, producerPos._2().intValue()); - } - - @Override - public void finish() { + assertEquals(lastSeenPos + 1, producerPos.getRight().intValue()); } @Override @@ -263,21 +256,15 @@ protected Integer getResult() { }); }).collect(Collectors.toList()); - // Close queue - Future closeFuture = executorService.submit(() -> { + // wait for all producers finished. + futureList.forEach(future -> { try { - for (Future f : futureList) { - f.get(); - } + future.get(); } catch (Exception e) { - throw new RuntimeException(e); + // ignore here } - return true; }); - // wait for producer finished - closeFuture.get(); - // wait for all the records consumed. queue.close(); queue.waitForConsumingFinished(); @@ -299,8 +286,8 @@ public void testException() throws Exception { final int numProducers = 40; final DisruptorMessageQueue queue = - new DisruptorMessageQueue(1024, getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA), - "BLOCKING_WAIT", numProducers, new Runnable() { + new DisruptorMessageQueue(Option.of(1024), getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA), + Option.of("BLOCKING_WAIT"), numProducers, new Runnable() { @Override public void run() { // do nothing. @@ -335,19 +322,15 @@ public void consumeOneRecord(HoodieLazyInsertIterable.HoodieInsertValueGenResult count++; } - @Override - public void finish() { - } - @Override protected Integer getResult() { return count; } }; - DisruptorExecutor>, Integer> exec = new DisruptorExecutor(1024, + DisruptorExecutor>, Integer> exec = new DisruptorExecutor(Option.of(1024), producers, Option.of(consumer), getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA), - WaitStrategyFactory.DEFAULT_STRATEGY, getPreExecuteRunnable()); + Option.of(WaitStrategyFactory.DEFAULT_STRATEGY), getPreExecuteRunnable()); final Throwable thrown = assertThrows(HoodieException.class, exec::execute, "exception is expected"); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryExecutor.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryExecutor.java index 747018123ecce..eb9467f93ab12 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryExecutor.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryExecutor.java @@ -22,11 +22,14 @@ import org.apache.hudi.common.util.Functions; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.SizeEstimator; +import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.exception.HoodieIOException; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; +import java.io.IOException; import java.util.Collections; import java.util.Iterator; import java.util.List; @@ -39,16 +42,13 @@ import java.util.stream.Collectors; /** - * Executor which orchestrates concurrent producers and consumers communicating through a bounded in-memory queue. This + * Executor which orchestrates concurrent producers and consumers communicating through 'BoundedInMemoryQueue'. This * class takes as input the size limit, queue producer(s), consumer and transformer and exposes API to orchestrate * concurrent execution of these actors communicating through a central bounded queue */ -public class BoundedInMemoryExecutor implements HoodieExecutor { +public class BoundedInMemoryExecutor extends MessageQueueBasedHoodieExecutor { private static final Logger LOG = LogManager.getLogger(BoundedInMemoryExecutor.class); - // Used for buffering records which is controlled by HoodieWriteConfig#WRITE_BUFFER_LIMIT_BYTES. - private final BoundedInMemoryQueue queue; - private final HoodieExecutorBase hoodieExecutorBase; public BoundedInMemoryExecutor(final long bufferLimitInBytes, final Iterator inputItr, IteratorBasedQueueConsumer consumer, Function transformFunction, Runnable preExecuteRunnable) { @@ -68,19 +68,19 @@ public BoundedInMemoryExecutor(final long bufferLimitInBytes, HoodieProducer public BoundedInMemoryExecutor(final long bufferLimitInBytes, List> producers, Option> consumer, final Function transformFunction, final SizeEstimator sizeEstimator, Runnable preExecuteRunnable) { - this.queue = new BoundedInMemoryQueue<>(bufferLimitInBytes, transformFunction, sizeEstimator); - this.hoodieExecutorBase = new HoodieExecutorBase<>(producers, consumer, preExecuteRunnable); + super(new BoundedInMemoryQueue<>(bufferLimitInBytes, transformFunction, sizeEstimator), + new HoodieExecutorBase<>(producers, consumer, preExecuteRunnable)); } /** * Start all Producers. */ - public ExecutorCompletionService startProducers() { + public List> startProducers() { // Latch to control when and which producer thread will close the queue final CountDownLatch latch = new CountDownLatch(hoodieExecutorBase.getProducers().size()); final ExecutorCompletionService completionService = new ExecutorCompletionService(hoodieExecutorBase.getProducerExecutorService()); - hoodieExecutorBase.getProducers().stream().map(producer -> { + List> producerTasks = hoodieExecutorBase.getProducers().stream().map(producer -> { return completionService.submit(() -> { try { hoodieExecutorBase.getPreExecuteRunnable().run(); @@ -101,7 +101,7 @@ public ExecutorCompletionService startProducers() { return true; }); }).collect(Collectors.toList()); - return completionService; + return producerTasks; } /** @@ -126,44 +126,30 @@ public Future startConsumer() { }).orElse(CompletableFuture.completedFuture(null)); } - /** - * Main API to run both production and consumption. - */ - public E execute() { - try { - startProducers(); - Future future = startConsumer(); - return finishConsuming(future); - } catch (InterruptedException ie) { - shutdownNow(); - Thread.currentThread().interrupt(); - throw new HoodieException(ie); - } catch (Exception e) { - throw new HoodieException(e); - } - } - public boolean isRemaining() { - return queue.iterator().hasNext(); + return ((BoundedInMemoryQueue)queue).iterator().hasNext(); } @Override - public E finishConsuming(Object o) throws ExecutionException, InterruptedException { - Future future = (Future) o; - return future.get(); + public void postAction() { + hoodieExecutorBase.close(); } public void shutdownNow() { hoodieExecutorBase.shutdownNow(); // close queue to force producer stop - queue.close(); + try { + queue.close(); + } catch (IOException e) { + throw new HoodieIOException("catch IOException while closing HoodieMessageQueue", e); + } } public boolean awaitTermination() { return hoodieExecutorBase.awaitTermination(); } - public BoundedInMemoryQueue getQueue() { + public HoodieMessageQueue getQueue() { return queue; } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryQueue.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryQueue.java index b23db3a259593..808181b585c37 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryQueue.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryQueue.java @@ -259,6 +259,11 @@ public void markAsFailed(Throwable e) { this.rateLimiter.release(RECORD_CACHING_LIMIT + 1); } + @Override + public boolean isEmpty() { + return this.queue.size() == 0; + } + @Override public Iterator iterator() { return iterator; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorExecutor.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorExecutor.java index 9769492e4b075..6b8ae9fa78350 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorExecutor.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorExecutor.java @@ -20,65 +20,62 @@ import org.apache.hudi.common.util.Functions; import org.apache.hudi.common.util.Option; -import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.exception.HoodieIOException; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; +import java.io.IOException; 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.Future; import java.util.function.Function; import java.util.stream.Collectors; -public class DisruptorExecutor implements HoodieExecutor { +/** + * Executor which orchestrates concurrent producers and consumers communicating through 'DisruptorMessageQueue'. This + * class takes as queue producer(s), consumer and transformer and exposes API to orchestrate + * concurrent execution of these actors communicating through disruptor + */ +public class DisruptorExecutor extends MessageQueueBasedHoodieExecutor { private static final Logger LOG = LogManager.getLogger(DisruptorExecutor.class); - // Used for buffering records which is controlled by HoodieWriteConfig#WRITE_BUFFER_LIMIT_BYTES. - private final DisruptorMessageQueue queue; - private final HoodieExecutorBase hoodieExecutorBase; - - public DisruptorExecutor(final int bufferSize, final Iterator inputItr, - IteratorBasedQueueConsumer consumer, Function transformFunction, String waitStrategy, Runnable preExecuteRunnable) { + public DisruptorExecutor(final Option bufferSize, final Iterator inputItr, + IteratorBasedQueueConsumer consumer, Function transformFunction, Option waitStrategy, Runnable preExecuteRunnable) { this(bufferSize, new IteratorBasedQueueProducer<>(inputItr), Option.of(consumer), transformFunction, waitStrategy, preExecuteRunnable); } - public DisruptorExecutor(final int bufferSize, HoodieProducer producer, + public DisruptorExecutor(final Option bufferSize, HoodieProducer producer, Option> consumer, final Function transformFunction) { - this(bufferSize, producer, consumer, transformFunction, WaitStrategyFactory.DEFAULT_STRATEGY, Functions.noop()); + this(bufferSize, producer, consumer, transformFunction, Option.of(WaitStrategyFactory.DEFAULT_STRATEGY), Functions.noop()); } - public DisruptorExecutor(final int bufferSize, HoodieProducer producer, - Option> consumer, final Function transformFunction, String waitStrategy, Runnable preExecuteRunnable) { + public DisruptorExecutor(final Option bufferSize, HoodieProducer producer, + Option> consumer, final Function transformFunction, Option waitStrategy, Runnable preExecuteRunnable) { this(bufferSize, Collections.singletonList(producer), consumer, transformFunction, waitStrategy, preExecuteRunnable); } - public DisruptorExecutor(final int bufferSize, List> producers, + public DisruptorExecutor(final Option bufferSize, List> producers, Option> consumer, final Function transformFunction, - final String waitStrategy, Runnable preExecuteRunnable) { - this.queue = new DisruptorMessageQueue<>(bufferSize, transformFunction, waitStrategy, producers.size(), preExecuteRunnable); - this.hoodieExecutorBase = new HoodieExecutorBase<>(producers, consumer, preExecuteRunnable); + final Option waitStrategy, Runnable preExecuteRunnable) { + super(new DisruptorMessageQueue<>(bufferSize, transformFunction, waitStrategy, producers.size(), preExecuteRunnable), + new HoodieExecutorBase<>(producers, consumer, preExecuteRunnable)); } /** * Start all Producers. */ - public ExecutorCompletionService startProducers() { + public List> startProducers() { final ExecutorCompletionService completionService = new ExecutorCompletionService(hoodieExecutorBase.getProducerExecutorService()); - hoodieExecutorBase.getProducers().stream().map(producer -> { + return hoodieExecutorBase.getProducers().stream().map(producer -> { return completionService.submit(() -> { try { - hoodieExecutorBase.getPreExecuteRunnable().run(); - - DisruptorPublisher publisher = new DisruptorPublisher<>(producer, queue); - publisher.startProduce(); - + producer.produce(queue); } catch (Throwable e) { LOG.error("error producing records", e); throw new HoodieException("Error producing records in disruptor executor", e); @@ -86,51 +83,40 @@ public ExecutorCompletionService startProducers() { return true; }); }).collect(Collectors.toList()); - return completionService; } @Override - public E execute() { - try { - ValidationUtils.checkState(hoodieExecutorBase.getConsumer().isPresent()); - startConsumer(); - ExecutorCompletionService pool = startProducers(); - return finishConsuming(pool); - } catch (InterruptedException ie) { - shutdownNow(); - Thread.currentThread().interrupt(); - throw new HoodieException(ie); - } catch (Exception e) { - throw new HoodieException(e); - } + public void setup() { + DisruptorMessageHandler handler = new DisruptorMessageHandler<>(hoodieExecutorBase.getConsumer().get()); + ((DisruptorMessageQueue)queue).setHandlers(handler); + ((DisruptorMessageQueue)queue).start(); } @Override - public E finishConsuming(Object o) throws ExecutionException, InterruptedException { - ExecutorCompletionService pool = (ExecutorCompletionService) o; - waitForProducersFinished(pool); - queue.close(); - queue.waitForConsumingFinished(); - hoodieExecutorBase.getConsumer().get().finish(); - - return hoodieExecutorBase.getConsumer().get().getResult(); - } - - private void waitForProducersFinished(ExecutorCompletionService pool) throws InterruptedException, ExecutionException { - for (int i = 0; i < hoodieExecutorBase.getProducers().size(); i++) { - pool.take().get(); + public void postAction() { + try { + queue.close(); + hoodieExecutorBase.close(); + } catch (IOException e) { + throw new HoodieIOException("Catch IOException while closing DisruptorMessageQueue", e); } } - /** - * Start only consumer. - */ @Override public Future startConsumer() { - DisruptorMessageHandler handler = new DisruptorMessageHandler<>(hoodieExecutorBase.getConsumer().get()); - queue.setHandlers(handler); - queue.start(); - return null; + // consumer is already started. Here just wait for all record consumed and return the result. + return hoodieExecutorBase.getConsumerExecutorService().submit(() -> { + + // Waits for all producers finished. + for (Future f : producerTasks) { + f.get(); + } + + // Call disruptor queue close function which will wait until all events currently in the disruptor have been processed by all event processors + queue.close(); + hoodieExecutorBase.getConsumer().get().finish(); + return hoodieExecutorBase.getConsumer().get().getResult(); + }); } @Override @@ -141,11 +127,15 @@ public boolean isRemaining() { @Override public void shutdownNow() { hoodieExecutorBase.shutdownNow(); - queue.close(); + try { + queue.close(); + } catch (IOException e) { + throw new HoodieIOException("Catch IOException while closing DisruptorMessageQueue"); + } } @Override - public DisruptorMessageQueue getQueue() { + public HoodieMessageQueue getQueue() { return queue; } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageQueue.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageQueue.java index 4845175cdcdc6..0877e77fb1e62 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageQueue.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageQueue.java @@ -37,11 +37,11 @@ public class DisruptorMessageQueue implements HoodieMessageQueue { private final Function transformFunction; private final RingBuffer> ringBuffer; - public DisruptorMessageQueue(int bufferSize, Function transformFunction, String waitStrategyName, int totalProducers, Runnable preExecuteRunnable) { + public DisruptorMessageQueue(Option bufferSize, Function transformFunction, Option waitStrategyName, int totalProducers, Runnable preExecuteRunnable) { WaitStrategy waitStrategy = WaitStrategyFactory.build(waitStrategyName); HoodieDaemonThreadFactory threadFactory = new HoodieDaemonThreadFactory(preExecuteRunnable); - this.queue = new Disruptor<>(HoodieDisruptorEvent::new, bufferSize, threadFactory, totalProducers > 1 ? ProducerType.MULTI : ProducerType.SINGLE, waitStrategy); + this.queue = new Disruptor<>(new HoodieDisruptorEventFactory<>(), bufferSize.get(), threadFactory, totalProducers > 1 ? ProducerType.MULTI : ProducerType.SINGLE, waitStrategy); this.ringBuffer = queue.getRingBuffer(); this.transformFunction = transformFunction; } @@ -71,8 +71,14 @@ public Option readNextRecord() { return null; } + @Override + public void markAsFailed(Throwable e) { + // do nothing. + } + @Override public void close() { + // Waits until all events currently in the disruptor have been processed by all event processors queue.shutdown(); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieDaemonThreadFactory.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieDaemonThreadFactory.java index 243f3a81e57b3..e5be5bcdacfd7 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieDaemonThreadFactory.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieDaemonThreadFactory.java @@ -18,8 +18,11 @@ package org.apache.hudi.common.util.queue; +import org.apache.hudi.common.util.Functions; + import java.util.concurrent.ThreadFactory; import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Function; /** * Access to a ThreadFactory instance. @@ -31,11 +34,23 @@ public class HoodieDaemonThreadFactory implements ThreadFactory { private final Runnable preExecuteRunnable; private final AtomicInteger threadsNum = new AtomicInteger(); private final String namePattern; - private final String baseName = "Hoodie-disruptor-daemon-thread"; + private static final String BASE_NAME = "Hoodie-daemon-thread"; + + public HoodieDaemonThreadFactory() { + this(BASE_NAME, Functions.noop()); + } + + public HoodieDaemonThreadFactory(String threadNamePrefix) { + this(threadNamePrefix, Functions.noop()); + } public HoodieDaemonThreadFactory(Runnable preExecuteRunnable) { + this(BASE_NAME, preExecuteRunnable); + } + + public HoodieDaemonThreadFactory(String threadNamePrefix, Runnable preExecuteRunnable) { this.preExecuteRunnable = preExecuteRunnable; - this.namePattern = baseName + "-%d"; + this.namePattern = threadNamePrefix + "-%d"; } @Override @@ -48,6 +63,7 @@ public void run() { r.run(); } }, String.format(namePattern, threadsNum.addAndGet(1))); + t.setDaemon(true); return t; } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieExecutor.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieExecutor.java index 4e1fccb2c6df5..22d5043f7b29f 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieExecutor.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieExecutor.java @@ -18,8 +18,7 @@ package org.apache.hudi.common.util.queue; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.ExecutorCompletionService; +import java.util.List; import java.util.concurrent.Future; /** @@ -30,7 +29,7 @@ public interface HoodieExecutor { /** * Start all Producers. */ - ExecutorCompletionService startProducers(); + List> startProducers(); /** * Start consumer. @@ -48,9 +47,9 @@ public interface HoodieExecutor { boolean isRemaining(); /** - * Wait for consumer finish consuming and return result. + * Closing/cleaning up the executor's resources after consuming finished. */ - E finishConsuming(Object o) throws ExecutionException, InterruptedException; + void postAction(); /** * Shutdown all the consumers and producers. @@ -63,4 +62,9 @@ public interface HoodieExecutor { HoodieMessageQueue getQueue(); boolean awaitTermination(); + + /** + * set all the resources for current HoodieExecutor before start to produce and consume records. + */ + void setup(); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieExecutorBase.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieExecutorBase.java index cb9f48c2c85da..d960fbd4768a8 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieExecutorBase.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieExecutorBase.java @@ -54,7 +54,7 @@ public HoodieExecutorBase(List> producers, Option extends Closeable { * Read records from inner message queue. */ Option readNextRecord(); + + /** + * API to allow producers and consumer to communicate termination due to failure. + */ + void markAsFailed(Throwable e); + + boolean isEmpty(); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/IteratorBasedQueueConsumer.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/IteratorBasedQueueConsumer.java index 5fde765432caf..4310be92d0c1d 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/IteratorBasedQueueConsumer.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/IteratorBasedQueueConsumer.java @@ -53,7 +53,7 @@ public O consume(HoodieMessageQueue queue) throws Exception { /** * Notifies implementation that we have exhausted consuming records from queue. */ - public abstract void finish(); + public void finish(){} /** * Return result of consuming records so far. diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/MessageQueueBasedHoodieExecutor.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/MessageQueueBasedHoodieExecutor.java new file mode 100644 index 0000000000000..3286c078942a1 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/MessageQueueBasedHoodieExecutor.java @@ -0,0 +1,69 @@ +/* + * 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.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.exception.HoodieException; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import java.util.List; +import java.util.concurrent.Future; + +/** + * MessageQueueBasedHoodieExecutor holds a inner message queue, producers and consumer. + * Producers produce message into message queue and consumer can consume message from this inner message queue. + */ +public abstract class MessageQueueBasedHoodieExecutor implements HoodieExecutor { + + private static final Logger LOG = LogManager.getLogger(MessageQueueBasedHoodieExecutor.class); + + public HoodieMessageQueue queue; + public HoodieExecutorBase hoodieExecutorBase; + public List> producerTasks; + + public MessageQueueBasedHoodieExecutor(HoodieMessageQueue queue, HoodieExecutorBase hoodieExecutorBase) { + this.hoodieExecutorBase = hoodieExecutorBase; + this.queue = queue; + } + + /** + * Main API to run both production and consumption. + */ + @Override + public E execute() { + try { + ValidationUtils.checkState(hoodieExecutorBase.getConsumer().isPresent()); + setup(); + producerTasks = startProducers(); + Future future = startConsumer(); + return future.get(); + } catch (InterruptedException ie) { + shutdownNow(); + Thread.currentThread().interrupt(); + throw new HoodieException(ie); + } catch (Exception e) { + throw new HoodieException(e); + } finally { + postAction(); + } + } + + public void setup(){} +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/WaitStrategyFactory.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/WaitStrategyFactory.java index a749cae886272..25fd1a069044c 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/WaitStrategyFactory.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/WaitStrategyFactory.java @@ -25,6 +25,7 @@ import com.lmax.disruptor.SleepingWaitStrategy; import com.lmax.disruptor.WaitStrategy; import com.lmax.disruptor.YieldingWaitStrategy; +import org.apache.hudi.common.util.Option; import org.apache.hudi.exception.HoodieException; public class WaitStrategyFactory { @@ -34,9 +35,12 @@ public class WaitStrategyFactory { /** * Build WaitStrategy for disruptor */ - public static WaitStrategy build(String name) { + public static WaitStrategy build(Option name) { + if (!name.isPresent()) { + return new BlockingWaitStrategy(); + } - DisruptorWaitStrategyType strategyType = DisruptorWaitStrategyType.valueOf(name.toUpperCase()); + DisruptorWaitStrategyType strategyType = DisruptorWaitStrategyType.valueOf(name.get().toUpperCase()); switch (strategyType) { case BLOCKING_WAIT: return new BlockingWaitStrategy(); From 270d811f5b97923137f6d4b1b5698e60c2b35986 Mon Sep 17 00:00:00 2001 From: yuezhang Date: Fri, 14 Oct 2022 17:05:58 +0800 Subject: [PATCH 34/50] address comments --- .../util/queue/DisruptorMessageQueue.java | 37 ++++++++++++++++ .../util/queue/HoodieDisruptorEvent.java | 43 ------------------- .../queue/HoodieDisruptorEventFactory.java | 33 -------------- 3 files changed, 37 insertions(+), 76 deletions(-) delete mode 100644 hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieDisruptorEvent.java delete mode 100644 hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieDisruptorEventFactory.java diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageQueue.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageQueue.java index 0877e77fb1e62..48b32b0b73e8a 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageQueue.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageQueue.java @@ -18,6 +18,7 @@ package org.apache.hudi.common.util.queue; +import com.lmax.disruptor.EventFactory; import com.lmax.disruptor.EventTranslator; import com.lmax.disruptor.RingBuffer; import com.lmax.disruptor.WaitStrategy; @@ -101,3 +102,39 @@ public void start() { queue.start(); } } + +/** + * HoodieDisruptorEventFactory is used to create/preallocate HoodieDisruptorEvent. + * + */ +class HoodieDisruptorEventFactory implements EventFactory> { + + @Override + public HoodieDisruptorEvent newInstance() { + return new HoodieDisruptorEvent<>(); + } +} + +/** + * The unit of data passed from producer to consumer in disruptor world. + */ +class HoodieDisruptorEvent { + + private O value; + + public void set(O value) { + this.value = value; + } + + public O get() { + return this.value; + } + + /** + * When passing data via the Disruptor, it is possible for objects to live longer than intended. + * To avoid this happening it is necessary to clear out the event after processing it. + */ + public void clear() { + value = null; + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieDisruptorEvent.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieDisruptorEvent.java deleted file mode 100644 index 9fd4e13addb16..0000000000000 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieDisruptorEvent.java +++ /dev/null @@ -1,43 +0,0 @@ -/* - * 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; - -/** - * The unit of data passed from producer to consumer in disruptor world. - */ -public class HoodieDisruptorEvent { - - private O value; - - public void set(O value) { - this.value = value; - } - - public O get() { - return this.value; - } - - /** - * When passing data via the Disruptor, it is possible for objects to live longer than intended. - * To avoid this happening it is necessary to clear out the event after processing it. - */ - public void clear() { - value = null; - } -} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieDisruptorEventFactory.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieDisruptorEventFactory.java deleted file mode 100644 index a9aa2a558c71e..0000000000000 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieDisruptorEventFactory.java +++ /dev/null @@ -1,33 +0,0 @@ -/* - * 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.EventFactory; - -/** - * HoodieDisruptorEventFactory is used to create/preallocate HoodieDisruptorEvent. - * - */ -public class HoodieDisruptorEventFactory implements EventFactory> { - - @Override - public HoodieDisruptorEvent newInstance() { - return new HoodieDisruptorEvent<>(); - } -} From 0fc24d3af4181d8fb68d803b97be78e5cd448787 Mon Sep 17 00:00:00 2001 From: yuezhang Date: Fri, 14 Oct 2022 17:43:11 +0800 Subject: [PATCH 35/50] address comments --- .../common/util/queue/BoundedInMemoryExecutor.java | 10 ++++------ .../hudi/common/util/queue/DisruptorExecutor.java | 4 ++-- 2 files changed, 6 insertions(+), 8 deletions(-) diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryExecutor.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryExecutor.java index eb9467f93ab12..2f11850368199 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryExecutor.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryExecutor.java @@ -22,7 +22,6 @@ import org.apache.hudi.common.util.Functions; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.SizeEstimator; -import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieIOException; @@ -35,7 +34,6 @@ import java.util.List; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CountDownLatch; -import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorCompletionService; import java.util.concurrent.Future; import java.util.function.Function; @@ -80,7 +78,8 @@ public List> startProducers() { final CountDownLatch latch = new CountDownLatch(hoodieExecutorBase.getProducers().size()); final ExecutorCompletionService completionService = new ExecutorCompletionService(hoodieExecutorBase.getProducerExecutorService()); - List> producerTasks = hoodieExecutorBase.getProducers().stream().map(producer -> { + + return hoodieExecutorBase.getProducers().stream().map(producer -> { return completionService.submit(() -> { try { hoodieExecutorBase.getPreExecuteRunnable().run(); @@ -101,7 +100,6 @@ public List> startProducers() { return true; }); }).collect(Collectors.toList()); - return producerTasks; } /** @@ -149,7 +147,7 @@ public boolean awaitTermination() { return hoodieExecutorBase.awaitTermination(); } - public HoodieMessageQueue getQueue() { - return queue; + public BoundedInMemoryQueue getQueue() { + return (BoundedInMemoryQueue)queue; } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorExecutor.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorExecutor.java index 6b8ae9fa78350..4aa046ee8248a 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorExecutor.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorExecutor.java @@ -135,8 +135,8 @@ public void shutdownNow() { } @Override - public HoodieMessageQueue getQueue() { - return queue; + public DisruptorMessageQueue getQueue() { + return (DisruptorMessageQueue)queue; } @Override From 47971661766fe9bd4c6bf4e01afabdd43345b844 Mon Sep 17 00:00:00 2001 From: yuezhang Date: Fri, 14 Oct 2022 18:02:02 +0800 Subject: [PATCH 36/50] address comments --- .../common/util/queue/DisruptorExecutor.java | 18 ++++---- .../util/queue/DisruptorMessageQueue.java | 41 ++++-------------- .../util/queue/HoodieDaemonThreadFactory.java | 1 - .../util/queue/HoodieDisruptorEvent.java | 43 +++++++++++++++++++ 4 files changed, 61 insertions(+), 42 deletions(-) create mode 100644 hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieDisruptorEvent.java diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorExecutor.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorExecutor.java index 4aa046ee8248a..3a805f73b32f4 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorExecutor.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorExecutor.java @@ -105,17 +105,17 @@ public void postAction() { @Override public Future startConsumer() { // consumer is already started. Here just wait for all record consumed and return the result. - return hoodieExecutorBase.getConsumerExecutorService().submit(() -> { + return hoodieExecutorBase.getConsumerExecutorService().submit(() -> { - // Waits for all producers finished. - for (Future f : producerTasks) { - f.get(); - } + // Waits for all producers finished. + for (Future f : producerTasks) { + f.get(); + } - // Call disruptor queue close function which will wait until all events currently in the disruptor have been processed by all event processors - queue.close(); - hoodieExecutorBase.getConsumer().get().finish(); - return hoodieExecutorBase.getConsumer().get().getResult(); + // Call disruptor queue close function which will wait until all events currently in the disruptor have been processed by all event processors + queue.close(); + hoodieExecutorBase.getConsumer().get().finish(); + return hoodieExecutorBase.getConsumer().get().getResult(); }); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageQueue.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageQueue.java index 48b32b0b73e8a..c7cc267eb7b17 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageQueue.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageQueue.java @@ -101,40 +101,17 @@ public void setHandlers(DisruptorMessageHandler handler) { public void start() { queue.start(); } -} - -/** - * HoodieDisruptorEventFactory is used to create/preallocate HoodieDisruptorEvent. - * - */ -class HoodieDisruptorEventFactory implements EventFactory> { - - @Override - public HoodieDisruptorEvent newInstance() { - return new HoodieDisruptorEvent<>(); - } -} - -/** - * The unit of data passed from producer to consumer in disruptor world. - */ -class HoodieDisruptorEvent { - - private O value; - - public void set(O value) { - this.value = value; - } - - public O get() { - return this.value; - } /** - * When passing data via the Disruptor, it is possible for objects to live longer than intended. - * To avoid this happening it is necessary to clear out the event after processing it. + * HoodieDisruptorEventFactory is used to create/preallocate HoodieDisruptorEvent. + * */ - public void clear() { - value = null; + class HoodieDisruptorEventFactory implements EventFactory> { + + @Override + public HoodieDisruptorEvent newInstance() { + return new HoodieDisruptorEvent<>(); + } } } + diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieDaemonThreadFactory.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieDaemonThreadFactory.java index e5be5bcdacfd7..a2625b5a03db3 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieDaemonThreadFactory.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieDaemonThreadFactory.java @@ -22,7 +22,6 @@ import java.util.concurrent.ThreadFactory; import java.util.concurrent.atomic.AtomicInteger; -import java.util.function.Function; /** * Access to a ThreadFactory instance. diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieDisruptorEvent.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieDisruptorEvent.java new file mode 100644 index 0000000000000..4b882e80c4e5d --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieDisruptorEvent.java @@ -0,0 +1,43 @@ +/* + * 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; + +/** + * The unit of data passed from producer to consumer in disruptor world. + */ +class HoodieDisruptorEvent { + + private O value; + + public void set(O value) { + this.value = value; + } + + public O get() { + return this.value; + } + + /** + * When passing data via the Disruptor, it is possible for objects to live longer than intended. + * To avoid this happening it is necessary to clear out the event after processing it. + */ + public void clear() { + value = null; + } +} From 1fa9ac745ede5e07b6a8dfd270f35ef98f7babc2 Mon Sep 17 00:00:00 2001 From: yuezhang Date: Fri, 14 Oct 2022 18:07:10 +0800 Subject: [PATCH 37/50] address comments --- .../apache/hudi/execution/TestDisruptorMessageQueue.java | 1 - .../hudi/common/util/queue/DisruptorMessageQueue.java | 7 ------- 2 files changed, 8 deletions(-) diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorMessageQueue.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorMessageQueue.java index efa876e8bb1af..5fb55cf9f6aab 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorMessageQueue.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorMessageQueue.java @@ -267,7 +267,6 @@ protected Integer getResult() { // wait for all the records consumed. queue.close(); - queue.waitForConsumingFinished(); for (int i = 0; i < numProducers; i++) { // Ensure we have seen all the records for each producers diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageQueue.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageQueue.java index c7cc267eb7b17..7dc7ad99991ef 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageQueue.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageQueue.java @@ -83,13 +83,6 @@ public void close() { queue.shutdown(); } - public void waitForConsumingFinished() throws InterruptedException { - while (!isEmpty()) { - Thread.sleep(1000); - LOG.info("Still waiting for consuming finished."); - } - } - public boolean isEmpty() { return ringBuffer.getBufferSize() == ringBuffer.remainingCapacity(); } From e230fbb946d8f33a4691129893e2559a3d8a1768 Mon Sep 17 00:00:00 2001 From: yuezhang Date: Fri, 14 Oct 2022 18:15:28 +0800 Subject: [PATCH 38/50] address comments --- .../apache/hudi/common/util/queue/BoundedInMemoryExecutor.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryExecutor.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryExecutor.java index 2f11850368199..8d6ef187729a4 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryExecutor.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryExecutor.java @@ -71,7 +71,7 @@ public BoundedInMemoryExecutor(final long bufferLimitInBytes, List> startProducers() { // Latch to control when and which producer thread will close the queue From 298f66d2842b1fa3ce9c487fd3d0f94eda4bd2b1 Mon Sep 17 00:00:00 2001 From: yuezhang Date: Mon, 17 Oct 2022 10:45:17 +0800 Subject: [PATCH 39/50] address comments --- .../util/queue/BoundedInMemoryExecutor.java | 29 ++++---- .../common/util/queue/DisruptorExecutor.java | 28 ++++---- .../common/util/queue/HoodieExecutorBase.java | 63 +++++++++-------- .../MessageQueueBasedHoodieExecutor.java | 69 ------------------- 4 files changed, 61 insertions(+), 128 deletions(-) delete mode 100644 hudi-common/src/main/java/org/apache/hudi/common/util/queue/MessageQueueBasedHoodieExecutor.java diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryExecutor.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryExecutor.java index 8d6ef187729a4..ee7d213e772b7 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryExecutor.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryExecutor.java @@ -44,9 +44,10 @@ * class takes as input the size limit, queue producer(s), consumer and transformer and exposes API to orchestrate * concurrent execution of these actors communicating through a central bounded queue */ -public class BoundedInMemoryExecutor extends MessageQueueBasedHoodieExecutor { +public class BoundedInMemoryExecutor extends HoodieExecutorBase { private static final Logger LOG = LogManager.getLogger(BoundedInMemoryExecutor.class); + private final HoodieMessageQueue queue; public BoundedInMemoryExecutor(final long bufferLimitInBytes, final Iterator inputItr, IteratorBasedQueueConsumer consumer, Function transformFunction, Runnable preExecuteRunnable) { @@ -66,8 +67,8 @@ public BoundedInMemoryExecutor(final long bufferLimitInBytes, HoodieProducer public BoundedInMemoryExecutor(final long bufferLimitInBytes, List> producers, Option> consumer, final Function transformFunction, final SizeEstimator sizeEstimator, Runnable preExecuteRunnable) { - super(new BoundedInMemoryQueue<>(bufferLimitInBytes, transformFunction, sizeEstimator), - new HoodieExecutorBase<>(producers, consumer, preExecuteRunnable)); + super(producers, consumer, preExecuteRunnable); + this.queue = new BoundedInMemoryQueue<>(bufferLimitInBytes, transformFunction, sizeEstimator); } /** @@ -75,14 +76,14 @@ public BoundedInMemoryExecutor(final long bufferLimitInBytes, List> startProducers() { // Latch to control when and which producer thread will close the queue - final CountDownLatch latch = new CountDownLatch(hoodieExecutorBase.getProducers().size()); + final CountDownLatch latch = new CountDownLatch(producers.size()); final ExecutorCompletionService completionService = - new ExecutorCompletionService(hoodieExecutorBase.getProducerExecutorService()); + new ExecutorCompletionService(producerExecutorService); - return hoodieExecutorBase.getProducers().stream().map(producer -> { + return producers.stream().map(producer -> { return completionService.submit(() -> { try { - hoodieExecutorBase.getPreExecuteRunnable().run(); + preExecuteRunnable.run(); producer.produce(queue); } catch (Throwable e) { LOG.error("error producing records", e); @@ -107,10 +108,10 @@ public List> startProducers() { */ @Override public Future startConsumer() { - return hoodieExecutorBase.getConsumer().map(consumer -> { - return hoodieExecutorBase.getConsumerExecutorService().submit(() -> { + return consumer.map(consumer -> { + return consumerExecutorService.submit(() -> { LOG.info("starting consumer thread"); - hoodieExecutorBase.getPreExecuteRunnable().run(); + preExecuteRunnable.run(); try { E result = consumer.consume(queue); LOG.info("Queue Consumption is done; notifying producer threads"); @@ -130,11 +131,11 @@ public boolean isRemaining() { @Override public void postAction() { - hoodieExecutorBase.close(); + super.close(); } public void shutdownNow() { - hoodieExecutorBase.shutdownNow(); + super.shutdownNow(); // close queue to force producer stop try { queue.close(); @@ -143,10 +144,6 @@ public void shutdownNow() { } } - public boolean awaitTermination() { - return hoodieExecutorBase.awaitTermination(); - } - public BoundedInMemoryQueue getQueue() { return (BoundedInMemoryQueue)queue; } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorExecutor.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorExecutor.java index 3a805f73b32f4..d46a58e1a2808 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorExecutor.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorExecutor.java @@ -40,9 +40,10 @@ * class takes as queue producer(s), consumer and transformer and exposes API to orchestrate * concurrent execution of these actors communicating through disruptor */ -public class DisruptorExecutor extends MessageQueueBasedHoodieExecutor { +public class DisruptorExecutor extends HoodieExecutorBase { private static final Logger LOG = LogManager.getLogger(DisruptorExecutor.class); + private final HoodieMessageQueue queue; public DisruptorExecutor(final Option bufferSize, final Iterator inputItr, IteratorBasedQueueConsumer consumer, Function transformFunction, Option waitStrategy, Runnable preExecuteRunnable) { @@ -62,8 +63,8 @@ public DisruptorExecutor(final Option bufferSize, HoodieProducer pro public DisruptorExecutor(final Option bufferSize, List> producers, Option> consumer, final Function transformFunction, final Option waitStrategy, Runnable preExecuteRunnable) { - super(new DisruptorMessageQueue<>(bufferSize, transformFunction, waitStrategy, producers.size(), preExecuteRunnable), - new HoodieExecutorBase<>(producers, consumer, preExecuteRunnable)); + super(producers, consumer, preExecuteRunnable); + this.queue = new DisruptorMessageQueue<>(bufferSize, transformFunction, waitStrategy, producers.size(), preExecuteRunnable); } /** @@ -71,8 +72,8 @@ public DisruptorExecutor(final Option bufferSize, List> startProducers() { final ExecutorCompletionService completionService = - new ExecutorCompletionService(hoodieExecutorBase.getProducerExecutorService()); - return hoodieExecutorBase.getProducers().stream().map(producer -> { + new ExecutorCompletionService(producerExecutorService); + return producers.stream().map(producer -> { return completionService.submit(() -> { try { producer.produce(queue); @@ -87,7 +88,7 @@ public List> startProducers() { @Override public void setup() { - DisruptorMessageHandler handler = new DisruptorMessageHandler<>(hoodieExecutorBase.getConsumer().get()); + DisruptorMessageHandler handler = new DisruptorMessageHandler<>(consumer.get()); ((DisruptorMessageQueue)queue).setHandlers(handler); ((DisruptorMessageQueue)queue).start(); } @@ -96,7 +97,7 @@ public void setup() { public void postAction() { try { queue.close(); - hoodieExecutorBase.close(); + super.close(); } catch (IOException e) { throw new HoodieIOException("Catch IOException while closing DisruptorMessageQueue", e); } @@ -105,7 +106,7 @@ public void postAction() { @Override public Future startConsumer() { // consumer is already started. Here just wait for all record consumed and return the result. - return hoodieExecutorBase.getConsumerExecutorService().submit(() -> { + return consumerExecutorService.submit(() -> { // Waits for all producers finished. for (Future f : producerTasks) { @@ -114,8 +115,8 @@ public Future startConsumer() { // Call disruptor queue close function which will wait until all events currently in the disruptor have been processed by all event processors queue.close(); - hoodieExecutorBase.getConsumer().get().finish(); - return hoodieExecutorBase.getConsumer().get().getResult(); + consumer.get().finish(); + return consumer.get().getResult(); }); } @@ -126,7 +127,7 @@ public boolean isRemaining() { @Override public void shutdownNow() { - hoodieExecutorBase.shutdownNow(); + super.shutdownNow(); try { queue.close(); } catch (IOException e) { @@ -138,9 +139,4 @@ public void shutdownNow() { public DisruptorMessageQueue getQueue() { return (DisruptorMessageQueue)queue; } - - @Override - public boolean awaitTermination() { - return hoodieExecutorBase.awaitTermination(); - } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieExecutorBase.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieExecutorBase.java index d960fbd4768a8..32e97e4e4c4c1 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieExecutorBase.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieExecutorBase.java @@ -20,33 +20,38 @@ import org.apache.hudi.common.util.CustomizedThreadFactory; import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.exception.HoodieException; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import java.util.List; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; +import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; /** * HoodieExecutorBase holds common elements producerExecutorService, consumerExecutorService, producers and consumer. * Also HoodieExecutorBase control the lifecycle of producerExecutorService and consumerExecutorService. */ -public class HoodieExecutorBase { +public abstract class HoodieExecutorBase implements HoodieExecutor { - private static final Logger LOG = LogManager.getLogger(DisruptorExecutor.class); + private static final Logger LOG = LogManager.getLogger(HoodieExecutorBase.class); private static final long TERMINATE_WAITING_TIME_SECS = 60L; // Executor service used for launching write thread. - private final ExecutorService producerExecutorService; + public final ExecutorService producerExecutorService; // Executor service used for launching read thread. - private final ExecutorService consumerExecutorService; + public final ExecutorService consumerExecutorService; // Producers - private final List> producers; + public final List> producers; // Consumer - private final Option> consumer; + public final Option> consumer; // pre-execute function to implement environment specific behavior before executors (producers/consumer) run - private final Runnable preExecuteRunnable; + public final Runnable preExecuteRunnable; + + public List> producerTasks; public HoodieExecutorBase(List> producers, Option> consumer, Runnable preExecuteRunnable) { @@ -59,26 +64,6 @@ public HoodieExecutorBase(List> producers, Option> getProducers() { - return producers; - } - - public Option> getConsumer() { - return consumer; - } - public boolean awaitTermination() { // if current thread has been interrupted before awaitTermination was called, we still give // executor a chance to proceeding. So clear the interrupt flag and reset it if needed before return. @@ -107,4 +92,28 @@ public void close() { producerExecutorService.shutdown(); consumerExecutorService.shutdown(); } + + /** + * Main API to run both production and consumption. + */ + @Override + public E execute() { + try { + ValidationUtils.checkState(this.consumer.isPresent()); + setup(); + producerTasks = startProducers(); + Future future = startConsumer(); + return future.get(); + } catch (InterruptedException ie) { + shutdownNow(); + Thread.currentThread().interrupt(); + throw new HoodieException(ie); + } catch (Exception e) { + throw new HoodieException(e); + } finally { + postAction(); + } + } + + public void setup(){} } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/MessageQueueBasedHoodieExecutor.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/MessageQueueBasedHoodieExecutor.java deleted file mode 100644 index 3286c078942a1..0000000000000 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/MessageQueueBasedHoodieExecutor.java +++ /dev/null @@ -1,69 +0,0 @@ -/* - * 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.apache.hudi.common.util.ValidationUtils; -import org.apache.hudi.exception.HoodieException; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; - -import java.util.List; -import java.util.concurrent.Future; - -/** - * MessageQueueBasedHoodieExecutor holds a inner message queue, producers and consumer. - * Producers produce message into message queue and consumer can consume message from this inner message queue. - */ -public abstract class MessageQueueBasedHoodieExecutor implements HoodieExecutor { - - private static final Logger LOG = LogManager.getLogger(MessageQueueBasedHoodieExecutor.class); - - public HoodieMessageQueue queue; - public HoodieExecutorBase hoodieExecutorBase; - public List> producerTasks; - - public MessageQueueBasedHoodieExecutor(HoodieMessageQueue queue, HoodieExecutorBase hoodieExecutorBase) { - this.hoodieExecutorBase = hoodieExecutorBase; - this.queue = queue; - } - - /** - * Main API to run both production and consumption. - */ - @Override - public E execute() { - try { - ValidationUtils.checkState(hoodieExecutorBase.getConsumer().isPresent()); - setup(); - producerTasks = startProducers(); - Future future = startConsumer(); - return future.get(); - } catch (InterruptedException ie) { - shutdownNow(); - Thread.currentThread().interrupt(); - throw new HoodieException(ie); - } catch (Exception e) { - throw new HoodieException(e); - } finally { - postAction(); - } - } - - public void setup(){} -} From 8ded3e8bb4a9702fabaa3ac18b63427d2b422014 Mon Sep 17 00:00:00 2001 From: yuezhang Date: Mon, 17 Oct 2022 11:30:23 +0800 Subject: [PATCH 40/50] address comments --- .../execution/TestDisruptorMessageQueue.java | 42 +++++++-------- .../common/util/queue/DisruptorExecutor.java | 3 +- .../util/queue/DisruptorMessageHandler.java | 36 ------------- .../util/queue/DisruptorMessageQueue.java | 52 +++++++++++++++---- .../util/queue/HoodieDisruptorEvent.java | 43 --------------- 5 files changed, 62 insertions(+), 114 deletions(-) delete mode 100644 hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageHandler.java delete mode 100644 hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieDisruptorEvent.java diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorMessageQueue.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorMessageQueue.java index 5fb55cf9f6aab..d354a885f46b4 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorMessageQueue.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorMessageQueue.java @@ -27,7 +27,6 @@ import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.Pair; -import org.apache.hudi.common.util.queue.DisruptorMessageHandler; import org.apache.hudi.common.util.queue.DisruptorMessageQueue; import org.apache.hudi.common.util.queue.DisruptorPublisher; import org.apache.hudi.common.util.queue.FunctionBasedQueueProducer; @@ -62,6 +61,7 @@ import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -222,29 +222,26 @@ public void run() { // setup consumer and start disruptor - DisruptorMessageHandler, Integer> handler = - new DisruptorMessageHandler<>(new IteratorBasedQueueConsumer, Integer>() { - @Override - public void consumeOneRecord(HoodieLazyInsertIterable.HoodieInsertValueGenResult payload) { - // Read recs and ensure we have covered all producer recs. - final HoodieRecord rec = payload.record; - Pair producerPos = keyToProducerAndIndexMap.get(rec.getRecordKey()); - Integer lastSeenPos = lastSeenMap.get(producerPos.getLeft()); - countMap.put(producerPos.getLeft(), countMap.get(producerPos.getLeft()) + 1); - lastSeenMap.put(producerPos.getLeft(), lastSeenPos + 1); - // Ensure we are seeing the next record generated - assertEquals(lastSeenPos + 1, producerPos.getRight().intValue()); - } - - @Override - protected Integer getResult() { - return 0; - } - }); + queue.setHandlers(new IteratorBasedQueueConsumer, Integer>() { + @Override + public void consumeOneRecord(HoodieLazyInsertIterable.HoodieInsertValueGenResult payload) { + // Read recs and ensure we have covered all producer recs. + final HoodieRecord rec = payload.record; + Pair producerPos = keyToProducerAndIndexMap.get(rec.getRecordKey()); + Integer lastSeenPos = lastSeenMap.get(producerPos.getLeft()); + countMap.put(producerPos.getLeft(), countMap.get(producerPos.getLeft()) + 1); + lastSeenMap.put(producerPos.getLeft(), lastSeenPos + 1); + // Ensure we are seeing the next record generated + assertEquals(lastSeenPos + 1, producerPos.getRight().intValue()); + } - queue.setHandlers(handler); + @Override + protected Integer getResult() { + return 0; + } + }); queue.start(); @@ -333,7 +330,6 @@ protected Integer getResult() { final Throwable thrown = assertThrows(HoodieException.class, exec::execute, "exception is expected"); - assertEquals("java.util.concurrent.ExecutionException: org.apache.hudi.exception.HoodieException: Error producing records in disruptor executor", - thrown.getMessage()); + assertTrue(thrown.getMessage().contains("Error producing records in disruptor executor")); } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorExecutor.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorExecutor.java index d46a58e1a2808..2b59680455eaa 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorExecutor.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorExecutor.java @@ -88,8 +88,7 @@ public List> startProducers() { @Override public void setup() { - DisruptorMessageHandler handler = new DisruptorMessageHandler<>(consumer.get()); - ((DisruptorMessageQueue)queue).setHandlers(handler); + ((DisruptorMessageQueue)queue).setHandlers(consumer.get()); ((DisruptorMessageQueue)queue).start(); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageHandler.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageHandler.java deleted file mode 100644 index f98108224c9d2..0000000000000 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageHandler.java +++ /dev/null @@ -1,36 +0,0 @@ -/* - * 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.EventHandler; - -public class DisruptorMessageHandler implements EventHandler> { - - private IteratorBasedQueueConsumer consumer; - - public DisruptorMessageHandler(IteratorBasedQueueConsumer consumer) { - this.consumer = consumer; - } - - @Override - public void onEvent(HoodieDisruptorEvent event, long sequence, boolean endOfBatch) { - consumer.consumeOneRecord(event.get()); - event.clear(); - } -} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageQueue.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageQueue.java index 7dc7ad99991ef..ce0e241fead28 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageQueue.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageQueue.java @@ -19,6 +19,7 @@ package org.apache.hudi.common.util.queue; import com.lmax.disruptor.EventFactory; +import com.lmax.disruptor.EventHandler; import com.lmax.disruptor.EventTranslator; import com.lmax.disruptor.RingBuffer; import com.lmax.disruptor.WaitStrategy; @@ -34,15 +35,15 @@ public class DisruptorMessageQueue implements HoodieMessageQueue { private static final Logger LOG = LogManager.getLogger(DisruptorMessageQueue.class); - private final Disruptor> queue; + private final Disruptor queue; private final Function transformFunction; - private final RingBuffer> ringBuffer; + private final RingBuffer ringBuffer; public DisruptorMessageQueue(Option bufferSize, Function transformFunction, Option waitStrategyName, int totalProducers, Runnable preExecuteRunnable) { WaitStrategy waitStrategy = WaitStrategyFactory.build(waitStrategyName); HoodieDaemonThreadFactory threadFactory = new HoodieDaemonThreadFactory(preExecuteRunnable); - this.queue = new Disruptor<>(new HoodieDisruptorEventFactory<>(), bufferSize.get(), threadFactory, totalProducers > 1 ? ProducerType.MULTI : ProducerType.SINGLE, waitStrategy); + this.queue = new Disruptor<>(new HoodieDisruptorEventFactory(), bufferSize.get(), threadFactory, totalProducers > 1 ? ProducerType.MULTI : ProducerType.SINGLE, waitStrategy); this.ringBuffer = queue.getRingBuffer(); this.transformFunction = transformFunction; } @@ -56,9 +57,9 @@ public long size() { public void insertRecord(I value) throws Exception { O applied = transformFunction.apply(value); - EventTranslator> translator = new EventTranslator>() { + EventTranslator translator = new EventTranslator() { @Override - public void translateTo(HoodieDisruptorEvent event, long sequence) { + public void translateTo(HoodieDisruptorEvent event, long sequence) { event.set(applied); } }; @@ -87,8 +88,15 @@ public boolean isEmpty() { return ringBuffer.getBufferSize() == ringBuffer.remainingCapacity(); } - public void setHandlers(DisruptorMessageHandler handler) { - queue.handleEventsWith(handler); + public void setHandlers(IteratorBasedQueueConsumer consumer) { + queue.handleEventsWith(new EventHandler() { + + @Override + public void onEvent(HoodieDisruptorEvent event, long sequence, boolean endOfBatch) throws Exception { + consumer.consumeOneRecord(event.get()); + event.clear(); + } + }); } public void start() { @@ -99,11 +107,35 @@ public void start() { * HoodieDisruptorEventFactory is used to create/preallocate HoodieDisruptorEvent. * */ - class HoodieDisruptorEventFactory implements EventFactory> { + class HoodieDisruptorEventFactory implements EventFactory { @Override - public HoodieDisruptorEvent newInstance() { - return new HoodieDisruptorEvent<>(); + public HoodieDisruptorEvent newInstance() { + return new HoodieDisruptorEvent(); + } + } + + /** + * The unit of data passed from producer to consumer in disruptor world. + */ + class HoodieDisruptorEvent { + + private O value; + + public void set(O value) { + this.value = value; + } + + public O get() { + return this.value; + } + + /** + * When passing data via the Disruptor, it is possible for objects to live longer than intended. + * To avoid this happening it is necessary to clear out the event after processing it. + */ + public void clear() { + value = null; } } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieDisruptorEvent.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieDisruptorEvent.java deleted file mode 100644 index 4b882e80c4e5d..0000000000000 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieDisruptorEvent.java +++ /dev/null @@ -1,43 +0,0 @@ -/* - * 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; - -/** - * The unit of data passed from producer to consumer in disruptor world. - */ -class HoodieDisruptorEvent { - - private O value; - - public void set(O value) { - this.value = value; - } - - public O get() { - return this.value; - } - - /** - * When passing data via the Disruptor, it is possible for objects to live longer than intended. - * To avoid this happening it is necessary to clear out the event after processing it. - */ - public void clear() { - value = null; - } -} From 794e30be06f056e60824952e06c7eb7de4eb50c8 Mon Sep 17 00:00:00 2001 From: yuezhang Date: Mon, 17 Oct 2022 11:35:28 +0800 Subject: [PATCH 41/50] address comments --- .../hudi/common/util/queue/DisruptorMessageQueue.java | 9 --------- 1 file changed, 9 deletions(-) diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageQueue.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageQueue.java index ce0e241fead28..a165321885711 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageQueue.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageQueue.java @@ -94,7 +94,6 @@ public void setHandlers(IteratorBasedQueueConsumer consumer) { @Override public void onEvent(HoodieDisruptorEvent event, long sequence, boolean endOfBatch) throws Exception { consumer.consumeOneRecord(event.get()); - event.clear(); } }); } @@ -129,14 +128,6 @@ public void set(O value) { public O get() { return this.value; } - - /** - * When passing data via the Disruptor, it is possible for objects to live longer than intended. - * To avoid this happening it is necessary to clear out the event after processing it. - */ - public void clear() { - value = null; - } } } From 25fcd5d326a3ab26815fa914ec4f8496143075cd Mon Sep 17 00:00:00 2001 From: yuezhang Date: Mon, 17 Oct 2022 15:25:45 +0800 Subject: [PATCH 42/50] address comments --- .../apache/hudi/execution/TestDisruptorExecutionInSpark.java | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorExecutionInSpark.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorExecutionInSpark.java index 1e075fad22126..586f42f986f10 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorExecutionInSpark.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorExecutionInSpark.java @@ -124,8 +124,6 @@ protected Integer getResult() { @Test public void testInterruptExecutor() { final List hoodieRecords = dataGen.generateInserts(instantTime, 100); - final Lock lock = new ReentrantLock(); - Condition condition = lock.newCondition(); HoodieWriteConfig hoodieWriteConfig = mock(HoodieWriteConfig.class); when(hoodieWriteConfig.getWriteBufferSize()).thenReturn(Option.of(1024)); @@ -135,7 +133,7 @@ public void testInterruptExecutor() { @Override public void consumeOneRecord(HoodieLazyInsertIterable.HoodieInsertValueGenResult record) { try { - condition.wait(); + Thread.currentThread().wait(-1); } catch (InterruptedException ie) { // ignore here } From 6e56bd1d5d3a07cde1b23c66985b4cb4cc85ee39 Mon Sep 17 00:00:00 2001 From: yuezhang Date: Mon, 17 Oct 2022 15:25:54 +0800 Subject: [PATCH 43/50] address comments --- .../apache/hudi/execution/TestDisruptorExecutionInSpark.java | 3 --- 1 file changed, 3 deletions(-) diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorExecutionInSpark.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorExecutionInSpark.java index 586f42f986f10..1f5cc747f8275 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorExecutionInSpark.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorExecutionInSpark.java @@ -39,9 +39,6 @@ import java.util.ArrayList; import java.util.List; -import java.util.concurrent.locks.Condition; -import java.util.concurrent.locks.Lock; -import java.util.concurrent.locks.ReentrantLock; import scala.Tuple2; From d615f1f02d90ce8a26c3a9c7c0fd479e89de3155 Mon Sep 17 00:00:00 2001 From: yuezhang Date: Mon, 17 Oct 2022 16:36:30 +0800 Subject: [PATCH 44/50] address comments --- .../org/apache/hudi/common/util/queue/HoodieExecutorBase.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieExecutorBase.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieExecutorBase.java index 32e97e4e4c4c1..830360f502a5d 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieExecutorBase.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieExecutorBase.java @@ -32,7 +32,7 @@ import java.util.concurrent.TimeUnit; /** - * HoodieExecutorBase holds common elements producerExecutorService, consumerExecutorService, producers and consumer. + * HoodieExecutorBase holds common elements producerExecutorService, consumerExecutorService, producers and a signle consumer. * Also HoodieExecutorBase control the lifecycle of producerExecutorService and consumerExecutorService. */ public abstract class HoodieExecutorBase implements HoodieExecutor { From c748320fb229a7a013f4710ac1a2d68e60175d2e Mon Sep 17 00:00:00 2001 From: yuezhang Date: Mon, 17 Oct 2022 18:18:33 +0800 Subject: [PATCH 45/50] address comments --- .../org/apache/hudi/common/util/queue/HoodieExecutorBase.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieExecutorBase.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieExecutorBase.java index 830360f502a5d..770719422ce64 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieExecutorBase.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieExecutorBase.java @@ -32,7 +32,7 @@ import java.util.concurrent.TimeUnit; /** - * HoodieExecutorBase holds common elements producerExecutorService, consumerExecutorService, producers and a signle consumer. + * HoodieExecutorBase holds common elements producerExecutorService, consumerExecutorService, producers and a single consumer. * Also HoodieExecutorBase control the lifecycle of producerExecutorService and consumerExecutorService. */ public abstract class HoodieExecutorBase implements HoodieExecutor { From 3f3a41a03b2fe436a4014e6e6627aa90b7cfd80b Mon Sep 17 00:00:00 2001 From: yuezhang Date: Tue, 18 Oct 2022 00:59:33 +0800 Subject: [PATCH 46/50] address comments --- .../hudi/execution/TestDisruptorExecutionInSpark.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorExecutionInSpark.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorExecutionInSpark.java index 1f5cc747f8275..3046035b653d0 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorExecutionInSpark.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorExecutionInSpark.java @@ -36,6 +36,7 @@ import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import java.util.ArrayList; import java.util.List; @@ -119,6 +120,7 @@ protected Integer getResult() { } @Test + @Timeout(value = 60) public void testInterruptExecutor() { final List hoodieRecords = dataGen.generateInserts(instantTime, 100); @@ -130,7 +132,7 @@ public void testInterruptExecutor() { @Override public void consumeOneRecord(HoodieLazyInsertIterable.HoodieInsertValueGenResult record) { try { - Thread.currentThread().wait(-1); + Thread.currentThread().wait(); } catch (InterruptedException ie) { // ignore here } @@ -152,8 +154,6 @@ protected Integer getResult() { assertTrue(Thread.interrupted()); } catch (Exception e) { // ignore here - } finally { - executor.shutdownNow(); } } } From fd19907bf3ff68c7826eb92ab1432b94f7d6e8df Mon Sep 17 00:00:00 2001 From: yuezhang Date: Fri, 21 Oct 2022 18:06:12 +0800 Subject: [PATCH 47/50] address comments --- .../main/java/org/apache/hudi/config/HoodieWriteConfig.java | 5 +++-- .../java/org/apache/hudi/util/QueueBasedExecutorFactory.java | 4 ++-- .../apache/hudi/execution/TestDisruptorExecutionInSpark.java | 2 +- 3 files changed, 6 insertions(+), 5 deletions(-) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java index 04788bed752ab..3698678ee1558 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java @@ -92,6 +92,7 @@ import java.util.function.Supplier; import java.util.stream.Collectors; +import static org.apache.hudi.common.util.queue.ExecutorType.BOUNDED_IN_MEMORY; import static org.apache.hudi.config.HoodieCleanConfig.CLEANER_POLICY; /** @@ -136,11 +137,11 @@ public class HoodieWriteConfig extends HoodieConfig { public static final ConfigProperty EXECUTOR_TYPE = ConfigProperty .key("hoodie.write.executor.type") - .defaultValue("BOUNDED_IN_MEMORY") + .defaultValue(BOUNDED_IN_MEMORY.name()) .withDocumentation("Set executor which orchestrates concurrent producers and consumers communicating through a message queue." + "default value is BOUNDED_IN_MEMORY which use a bounded in-memory queue using LinkedBlockingQueue." + "Also users could use DISRUPTOR, which use disruptor as a lock free message queue " - + "to gain better writing performance. Although DISRUPTOR_EXECUTOR is still an experimental feature."); + + "to gain better writing performance if lock was the bottleneck. Although DISRUPTOR_EXECUTOR is still an experimental feature."); public static final ConfigProperty KEYGENERATOR_TYPE = ConfigProperty .key("hoodie.datasource.write.keygenerator.type") diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/util/QueueBasedExecutorFactory.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/util/QueueBasedExecutorFactory.java index e5e8be9d14ea9..722d13cf604ec 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/util/QueueBasedExecutorFactory.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/util/QueueBasedExecutorFactory.java @@ -34,8 +34,8 @@ public class QueueBasedExecutorFactory { /** * Create a new hoodie executor instance on demand. */ - public static HoodieExecutor create(HoodieWriteConfig hoodieConfig, Iterator inputItr, IteratorBasedQueueConsumer consumer, - Function transformFunction, Runnable preExecuteRunnable) { + public static HoodieExecutor create(HoodieWriteConfig hoodieConfig, Iterator inputItr, IteratorBasedQueueConsumer consumer, + Function transformFunction, Runnable preExecuteRunnable) { ExecutorType executorType = hoodieConfig.getExecutorType(); switch (executorType) { diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorExecutionInSpark.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorExecutionInSpark.java index 3046035b653d0..4a61c89d9b139 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorExecutionInSpark.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorExecutionInSpark.java @@ -107,7 +107,7 @@ protected Integer getResult() { // collect all records and assert that consumed records are identical to produced ones // assert there's no tampering, and that the ordering is preserved - assertEquals(hoodieRecords.size(), consumedRecords.size()); + assertEquals(hoodieRecords, consumedRecords); for (int i = 0; i < hoodieRecords.size(); i++) { assertEquals(hoodieRecords.get(i), consumedRecords.get(i)); } From e5c17f060235551dd9130e7bc7bbc33b294ebb18 Mon Sep 17 00:00:00 2001 From: yuezhang Date: Mon, 24 Oct 2022 12:24:31 +0800 Subject: [PATCH 48/50] address comments --- .../execution/TestBoundedInMemoryQueue.java | 22 ++++---- .../execution/TestDisruptorMessageQueue.java | 41 +++++++-------- .../common/util/ParquetReaderIterator.java | 4 +- .../util/queue/BoundedInMemoryExecutor.java | 51 +++++++++++-------- ...java => BoundedInMemoryQueueIterable.java} | 10 ++-- .../common/util/queue/DisruptorExecutor.java | 48 ++++++++--------- .../util/queue/DisruptorMessageQueue.java | 12 +---- .../common/util/queue/DisruptorPublisher.java | 34 ------------- .../common/util/queue/HoodieExecutor.java | 30 +---------- .../common/util/queue/HoodieExecutorBase.java | 43 +++++++++++++--- ...e.java => HoodieIterableMessageQueue.java} | 2 +- .../queue/IteratorBasedQueueConsumer.java | 2 +- .../util/queue/WaitStrategyFactory.java | 5 +- 13 files changed, 128 insertions(+), 176 deletions(-) rename hudi-common/src/main/java/org/apache/hudi/common/util/queue/{BoundedInMemoryQueue.java => BoundedInMemoryQueueIterable.java} (96%) delete mode 100644 hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorPublisher.java rename hudi-common/src/main/java/org/apache/hudi/common/util/queue/{IteratorBasedHoodieMessageQueue.java => HoodieIterableMessageQueue.java} (90%) diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestBoundedInMemoryQueue.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestBoundedInMemoryQueue.java index b052f848ea5c8..c36554bb64a7e 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestBoundedInMemoryQueue.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestBoundedInMemoryQueue.java @@ -26,7 +26,7 @@ import org.apache.hudi.common.util.FileIOUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.SizeEstimator; -import org.apache.hudi.common.util.queue.BoundedInMemoryQueue; +import org.apache.hudi.common.util.queue.BoundedInMemoryQueueIterable; import org.apache.hudi.common.util.queue.FunctionBasedQueueProducer; import org.apache.hudi.common.util.queue.HoodieProducer; import org.apache.hudi.common.util.queue.IteratorBasedQueueProducer; @@ -83,8 +83,8 @@ public void tearDown() throws Exception { public void testRecordReading() throws Exception { final int numRecords = 128; final List hoodieRecords = dataGen.generateInserts(instantTime, numRecords); - final BoundedInMemoryQueue queue = - new BoundedInMemoryQueue(FileIOUtils.KB, getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA)); + final BoundedInMemoryQueueIterable queue = + new BoundedInMemoryQueueIterable(FileIOUtils.KB, getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA)); // Produce Future resFuture = executorService.submit(() -> { new IteratorBasedQueueProducer<>(hoodieRecords.iterator()).produce(queue); @@ -123,8 +123,8 @@ public void testCompositeProducerRecordReading() throws Exception { final int numProducers = 40; final List> recs = new ArrayList<>(); - final BoundedInMemoryQueue queue = - new BoundedInMemoryQueue(FileIOUtils.KB, getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA)); + final BoundedInMemoryQueueIterable queue = + new BoundedInMemoryQueueIterable(FileIOUtils.KB, getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA)); // Record Key to Map> keyToProducerAndIndexMap = new HashMap<>(); @@ -222,8 +222,8 @@ public void testMemoryLimitForBuffering() throws Exception { getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA).apply((HoodieAvroRecord) hoodieRecords.get(0)); final long objSize = sizeEstimator.sizeEstimate(payload); final long memoryLimitInBytes = recordLimit * objSize; - final BoundedInMemoryQueue queue = - new BoundedInMemoryQueue(memoryLimitInBytes, getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA)); + final BoundedInMemoryQueueIterable queue = + new BoundedInMemoryQueueIterable(memoryLimitInBytes, getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA)); // Produce executorService.submit(() -> { @@ -275,8 +275,8 @@ public void testException() throws Exception { // first let us throw exception from queueIterator reader and test that queueing thread // stops and throws // correct exception back. - BoundedInMemoryQueue>> queue1 = - new BoundedInMemoryQueue(memoryLimitInBytes, getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA)); + BoundedInMemoryQueueIterable>> queue1 = + new BoundedInMemoryQueueIterable(memoryLimitInBytes, getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA)); // Produce Future resFuture = executorService.submit(() -> { @@ -303,8 +303,8 @@ public void testException() throws Exception { final Iterator mockHoodieRecordsIterator = mock(Iterator.class); when(mockHoodieRecordsIterator.hasNext()).thenReturn(true); when(mockHoodieRecordsIterator.next()).thenThrow(expectedException); - BoundedInMemoryQueue>> queue2 = - new BoundedInMemoryQueue(memoryLimitInBytes, getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA)); + BoundedInMemoryQueueIterable>> queue2 = + new BoundedInMemoryQueueIterable(memoryLimitInBytes, getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA)); // Produce Future res = executorService.submit(() -> { diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorMessageQueue.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorMessageQueue.java index d354a885f46b4..4ef76c34e4f9c 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorMessageQueue.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorMessageQueue.java @@ -28,7 +28,6 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.common.util.queue.DisruptorMessageQueue; -import org.apache.hudi.common.util.queue.DisruptorPublisher; import org.apache.hudi.common.util.queue.FunctionBasedQueueProducer; import org.apache.hudi.common.util.queue.HoodieProducer; import org.apache.hudi.common.util.queue.IteratorBasedQueueConsumer; @@ -51,7 +50,7 @@ import java.util.Iterator; import java.util.List; import java.util.Map; -import java.util.concurrent.Future; +import java.util.concurrent.CompletableFuture; import java.util.function.Function; import java.util.stream.Collectors; import java.util.stream.IntStream; @@ -191,15 +190,15 @@ public void run() { recs.add(pRecs); } - List disruptorPublishers = new ArrayList<>(); + List producers = new ArrayList<>(); for (int i = 0; i < recs.size(); i++) { final List r = recs.get(i); // Alternate between pull and push based iterators if (i % 2 == 0) { - DisruptorPublisher publisher = new DisruptorPublisher<>(new IteratorBasedQueueProducer<>(r.iterator()), queue); - disruptorPublishers.add(publisher); + HoodieProducer producer = new IteratorBasedQueueProducer<>(r.iterator()); + producers.add(producer); } else { - DisruptorPublisher publisher = new DisruptorPublisher<>(new FunctionBasedQueueProducer<>((buf) -> { + HoodieProducer producer = new FunctionBasedQueueProducer<>((buf) -> { Iterator itr = r.iterator(); while (itr.hasNext()) { try { @@ -209,8 +208,8 @@ public void run() { } } return true; - }), queue); - disruptorPublishers.add(publisher); + }); + producers.add(producer); } } @@ -220,9 +219,7 @@ public void run() { Map countMap = IntStream.range(0, numProducers).boxed().collect(Collectors.toMap(Function.identity(), x -> 0)); - // setup consumer and start disruptor - queue.setHandlers(new IteratorBasedQueueConsumer, Integer>() { @Override @@ -244,23 +241,19 @@ protected Integer getResult() { }); queue.start(); - // start to produce records - final List> futureList = disruptorPublishers.stream().map(disruptorPublisher -> { - return executorService.submit(() -> { - disruptorPublisher.startProduce(); + CompletableFuture producerFuture = CompletableFuture.allOf(producers.stream().map(producer -> { + return CompletableFuture.supplyAsync(() -> { + try { + producer.produce(queue); + } catch (Throwable e) { + throw new HoodieException("Error producing records in disruptor executor", e); + } return true; - }); - }).collect(Collectors.toList()); + }, executorService); + }).toArray(CompletableFuture[]::new)); - // wait for all producers finished. - futureList.forEach(future -> { - try { - future.get(); - } catch (Exception e) { - // ignore here - } - }); + producerFuture.get(); // wait for all the records consumed. queue.close(); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/ParquetReaderIterator.java b/hudi-common/src/main/java/org/apache/hudi/common/util/ParquetReaderIterator.java index 03bd471b606f1..347bcdf77a6ac 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/ParquetReaderIterator.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/ParquetReaderIterator.java @@ -18,7 +18,7 @@ package org.apache.hudi.common.util; -import org.apache.hudi.common.util.queue.BoundedInMemoryQueue; +import org.apache.hudi.common.util.queue.BoundedInMemoryQueueIterable; import org.apache.hudi.exception.HoodieException; import org.apache.parquet.hadoop.ParquetReader; @@ -27,7 +27,7 @@ /** * This class wraps a parquet reader and provides an iterator based api to read from a parquet file. This is used in - * {@link BoundedInMemoryQueue} + * {@link BoundedInMemoryQueueIterable} */ public class ParquetReaderIterator implements ClosableIterator { diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryExecutor.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryExecutor.java index ee7d213e772b7..f7efc3e65a4ca 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryExecutor.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryExecutor.java @@ -34,10 +34,7 @@ import java.util.List; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CountDownLatch; -import java.util.concurrent.ExecutorCompletionService; -import java.util.concurrent.Future; import java.util.function.Function; -import java.util.stream.Collectors; /** * Executor which orchestrates concurrent producers and consumers communicating through 'BoundedInMemoryQueue'. This @@ -68,20 +65,19 @@ public BoundedInMemoryExecutor(final long bufferLimitInBytes, List> consumer, final Function transformFunction, final SizeEstimator sizeEstimator, Runnable preExecuteRunnable) { super(producers, consumer, preExecuteRunnable); - this.queue = new BoundedInMemoryQueue<>(bufferLimitInBytes, transformFunction, sizeEstimator); + this.queue = new BoundedInMemoryQueueIterable<>(bufferLimitInBytes, transformFunction, sizeEstimator); } /** * Start all producers at once. */ - public List> startProducers() { + @Override + public CompletableFuture startProducers() { // Latch to control when and which producer thread will close the queue final CountDownLatch latch = new CountDownLatch(producers.size()); - final ExecutorCompletionService completionService = - new ExecutorCompletionService(producerExecutorService); - return producers.stream().map(producer -> { - return completionService.submit(() -> { + return CompletableFuture.allOf(producers.stream().map(producer -> { + return CompletableFuture.supplyAsync(() -> { try { preExecuteRunnable.run(); producer.produce(queue); @@ -94,22 +90,26 @@ public List> startProducers() { latch.countDown(); if (latch.getCount() == 0) { // Mark production as done so that consumer will be able to exit - queue.close(); + try { + queue.close(); + } catch (IOException e) { + throw new HoodieIOException("Catch Exception when closing BoundedInMemoryQueue.", e); + } } } } return true; - }); - }).collect(Collectors.toList()); + }, producerExecutorService); + }).toArray(CompletableFuture[]::new)); } /** * Start only consumer. */ @Override - public Future startConsumer() { + protected CompletableFuture startConsumer() { return consumer.map(consumer -> { - return consumerExecutorService.submit(() -> { + return CompletableFuture.supplyAsync(() -> { LOG.info("starting consumer thread"); preExecuteRunnable.run(); try { @@ -119,23 +119,26 @@ public Future startConsumer() { } catch (Exception e) { LOG.error("error consuming records", e); queue.markAsFailed(e); - throw e; + throw new HoodieException(e); } - }); + }, consumerExecutorService); }).orElse(CompletableFuture.completedFuture(null)); } + @Override public boolean isRemaining() { - return ((BoundedInMemoryQueue)queue).iterator().hasNext(); + return ((BoundedInMemoryQueueIterable)queue).iterator().hasNext(); } @Override - public void postAction() { + protected void postAction() { super.close(); } + @Override public void shutdownNow() { - super.shutdownNow(); + producerExecutorService.shutdownNow(); + consumerExecutorService.shutdownNow(); // close queue to force producer stop try { queue.close(); @@ -144,7 +147,13 @@ public void shutdownNow() { } } - public BoundedInMemoryQueue getQueue() { - return (BoundedInMemoryQueue)queue; + @Override + public BoundedInMemoryQueueIterable getQueue() { + return (BoundedInMemoryQueueIterable)queue; + } + + @Override + protected void setup() { + // do nothing. } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryQueue.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryQueueIterable.java similarity index 96% rename from hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryQueue.java rename to hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryQueueIterable.java index 808181b585c37..a0fc133391d17 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryQueue.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryQueueIterable.java @@ -49,7 +49,7 @@ * @param input payload data type * @param output payload data type */ -public class BoundedInMemoryQueue extends IteratorBasedHoodieMessageQueue { +public class BoundedInMemoryQueueIterable extends HoodieIterableMessageQueue { /** Interval used for polling records in the queue. **/ public static final int RECORD_POLL_INTERVAL_SEC = 1; @@ -60,7 +60,7 @@ public class BoundedInMemoryQueue extends IteratorBasedHoodieMessageQueue< /** Maximum records that will be cached. **/ private static final int RECORD_CACHING_LIMIT = 128 * 1024; - private static final Logger LOG = LogManager.getLogger(BoundedInMemoryQueue.class); + private static final Logger LOG = LogManager.getLogger(BoundedInMemoryQueueIterable.class); /** * It indicates number of records to cache. We will be using sampled record's average size to @@ -116,7 +116,7 @@ public class BoundedInMemoryQueue extends IteratorBasedHoodieMessageQueue< * @param memoryLimit MemoryLimit in bytes * @param transformFunction Transformer Function to convert input payload type to stored payload type */ - public BoundedInMemoryQueue(final long memoryLimit, final Function transformFunction) { + public BoundedInMemoryQueueIterable(final long memoryLimit, final Function transformFunction) { this(memoryLimit, transformFunction, new DefaultSizeEstimator() {}); } @@ -127,8 +127,8 @@ public BoundedInMemoryQueue(final long memoryLimit, final Function transfo * @param transformFunction Transformer Function to convert input payload type to stored payload type * @param payloadSizeEstimator Payload Size Estimator */ - public BoundedInMemoryQueue(final long memoryLimit, final Function transformFunction, - final SizeEstimator payloadSizeEstimator) { + public BoundedInMemoryQueueIterable(final long memoryLimit, final Function transformFunction, + final SizeEstimator payloadSizeEstimator) { this.memoryLimit = memoryLimit; this.transformFunction = transformFunction; this.payloadSizeEstimator = payloadSizeEstimator; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorExecutor.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorExecutor.java index 2b59680455eaa..13257d884d7ae 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorExecutor.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorExecutor.java @@ -30,10 +30,8 @@ import java.util.Collections; import java.util.Iterator; import java.util.List; -import java.util.concurrent.ExecutorCompletionService; -import java.util.concurrent.Future; +import java.util.concurrent.CompletableFuture; import java.util.function.Function; -import java.util.stream.Collectors; /** * Executor which orchestrates concurrent producers and consumers communicating through 'DisruptorMessageQueue'. This @@ -70,11 +68,10 @@ public DisruptorExecutor(final Option bufferSize, List> startProducers() { - final ExecutorCompletionService completionService = - new ExecutorCompletionService(producerExecutorService); - return producers.stream().map(producer -> { - return completionService.submit(() -> { + @Override + public CompletableFuture startProducers() { + return CompletableFuture.allOf(producers.stream().map(producer -> { + return CompletableFuture.supplyAsync(() -> { try { producer.produce(queue); } catch (Throwable e) { @@ -82,18 +79,18 @@ public List> startProducers() { throw new HoodieException("Error producing records in disruptor executor", e); } return true; - }); - }).collect(Collectors.toList()); + }, producerExecutorService); + }).toArray(CompletableFuture[]::new)); } @Override - public void setup() { + protected void setup() { ((DisruptorMessageQueue)queue).setHandlers(consumer.get()); ((DisruptorMessageQueue)queue).start(); } @Override - public void postAction() { + protected void postAction() { try { queue.close(); super.close(); @@ -103,20 +100,16 @@ public void postAction() { } @Override - public Future startConsumer() { - // consumer is already started. Here just wait for all record consumed and return the result. - return consumerExecutorService.submit(() -> { - - // Waits for all producers finished. - for (Future f : producerTasks) { - f.get(); + protected CompletableFuture startConsumer() { + return producerFuture.thenApplyAsync(res -> { + try { + queue.close(); + consumer.get().finish(); + return consumer.get().getResult(); + } catch (IOException e) { + throw new HoodieIOException("Catch Exception when closing", e); } - - // Call disruptor queue close function which will wait until all events currently in the disruptor have been processed by all event processors - queue.close(); - consumer.get().finish(); - return consumer.get().getResult(); - }); + }, consumerExecutorService); } @Override @@ -126,7 +119,8 @@ public boolean isRemaining() { @Override public void shutdownNow() { - super.shutdownNow(); + producerExecutorService.shutdownNow(); + consumerExecutorService.shutdownNow(); try { queue.close(); } catch (IOException e) { @@ -136,6 +130,6 @@ public void shutdownNow() { @Override public DisruptorMessageQueue getQueue() { - return (DisruptorMessageQueue)queue; + return (DisruptorMessageQueue)queue; } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageQueue.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageQueue.java index a165321885711..975bfa8dbbcf5 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageQueue.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageQueue.java @@ -56,21 +56,13 @@ public long size() { @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); - } - }; - + EventTranslator translator = (event, sequence) -> event.set(applied); queue.getRingBuffer().publishEvent(translator); } @Override public Option readNextRecord() { - // Let DisruptorMessageHandler to handle consuming logic. - return null; + throw new UnsupportedOperationException("Should not call readNextRecord here. And let DisruptorMessageHandler to handle consuming logic"); } @Override diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorPublisher.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorPublisher.java deleted file mode 100644 index accc633772765..0000000000000 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorPublisher.java +++ /dev/null @@ -1,34 +0,0 @@ -/* - * 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 DisruptorPublisher { - - HoodieProducer producer; - private final DisruptorMessageQueue queue; - - public DisruptorPublisher(HoodieProducer producer, DisruptorMessageQueue queue) { - this.producer = producer; - this.queue = queue; - } - - public void startProduce() throws Exception { - producer.produce(queue); - } -} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieExecutor.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieExecutor.java index 22d5043f7b29f..7d51441edec29 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieExecutor.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieExecutor.java @@ -18,23 +18,12 @@ package org.apache.hudi.common.util.queue; -import java.util.List; -import java.util.concurrent.Future; +import java.io.Closeable; /** * HoodieExecutor which orchestrates concurrent producers and consumers communicating through a bounded in message queue. */ -public interface HoodieExecutor { - - /** - * Start all Producers. - */ - List> startProducers(); - - /** - * Start consumer. - */ - Future startConsumer(); +public interface HoodieExecutor extends Closeable { /** * Main API to @@ -46,25 +35,10 @@ public interface HoodieExecutor { boolean isRemaining(); - /** - * Closing/cleaning up the executor's resources after consuming finished. - */ - void postAction(); - /** * Shutdown all the consumers and producers. */ void shutdownNow(); - /** - * get bounded in message queue. - */ - HoodieMessageQueue getQueue(); - boolean awaitTermination(); - - /** - * set all the resources for current HoodieExecutor before start to produce and consume records. - */ - void setup(); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieExecutorBase.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieExecutorBase.java index 770719422ce64..e123924ec8c68 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieExecutorBase.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieExecutorBase.java @@ -26,9 +26,9 @@ import org.apache.log4j.Logger; import java.util.List; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; -import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; /** @@ -51,7 +51,7 @@ public abstract class HoodieExecutorBase implements HoodieExecutor> producerTasks; + public CompletableFuture producerFuture; public HoodieExecutorBase(List> producers, Option> consumer, Runnable preExecuteRunnable) { @@ -59,11 +59,36 @@ public HoodieExecutorBase(List> producers, Option startProducers(); + + /** + * Start consumer. + */ + protected abstract CompletableFuture startConsumer(); + + /** + * Closing/cleaning up the executor's resources after consuming finished. + */ + protected abstract void postAction(); + + /** + * get bounded in message queue. + */ + public abstract HoodieMessageQueue getQueue(); + + /** + * set all the resources for current HoodieExecutor before start to produce and consume records. + */ + protected abstract void setup(); + public boolean awaitTermination() { // if current thread has been interrupted before awaitTermination was called, we still give // executor a chance to proceeding. So clear the interrupt flag and reset it if needed before return. @@ -83,11 +108,15 @@ public boolean awaitTermination() { return producerTerminated && consumerTerminated; } + /** + * Shutdown all the consumers and producers. + */ public void shutdownNow() { producerExecutorService.shutdownNow(); consumerExecutorService.shutdownNow(); } + @Override public void close() { producerExecutorService.shutdown(); consumerExecutorService.shutdown(); @@ -101,8 +130,8 @@ public E execute() { try { ValidationUtils.checkState(this.consumer.isPresent()); setup(); - producerTasks = startProducers(); - Future future = startConsumer(); + producerFuture = startProducers(); + CompletableFuture future = startConsumer(); return future.get(); } catch (InterruptedException ie) { shutdownNow(); @@ -114,6 +143,4 @@ public E execute() { postAction(); } } - - public void setup(){} } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/IteratorBasedHoodieMessageQueue.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieIterableMessageQueue.java similarity index 90% rename from hudi-common/src/main/java/org/apache/hudi/common/util/queue/IteratorBasedHoodieMessageQueue.java rename to hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieIterableMessageQueue.java index 06edcd66d6da4..71ef39f2c1883 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/IteratorBasedHoodieMessageQueue.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieIterableMessageQueue.java @@ -23,7 +23,7 @@ /** * IteratorBasedHoodieMessageQueue implements HoodieMessageQueue with Iterable */ -public abstract class IteratorBasedHoodieMessageQueue implements HoodieMessageQueue, Iterable { +public abstract class HoodieIterableMessageQueue implements HoodieMessageQueue, Iterable { public abstract Iterator iterator(); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/IteratorBasedQueueConsumer.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/IteratorBasedQueueConsumer.java index 4310be92d0c1d..713d6504645e0 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/IteratorBasedQueueConsumer.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/IteratorBasedQueueConsumer.java @@ -33,7 +33,7 @@ public abstract class IteratorBasedQueueConsumer implements HoodieConsumer @Override public O consume(HoodieMessageQueue queue) throws Exception { - Iterator iterator = ((IteratorBasedHoodieMessageQueue) queue).iterator(); + Iterator iterator = ((HoodieIterableMessageQueue) queue).iterator(); while (iterator.hasNext()) { consumeOneRecord(iterator.next()); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/WaitStrategyFactory.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/WaitStrategyFactory.java index 25fd1a069044c..8137d2a1360e9 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/WaitStrategyFactory.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/WaitStrategyFactory.java @@ -36,11 +36,8 @@ public class WaitStrategyFactory { * Build WaitStrategy for disruptor */ public static WaitStrategy build(Option name) { - if (!name.isPresent()) { - return new BlockingWaitStrategy(); - } + DisruptorWaitStrategyType strategyType = name.isPresent() ? DisruptorWaitStrategyType.valueOf(name.get().toUpperCase()) : BLOCKING_WAIT; - DisruptorWaitStrategyType strategyType = DisruptorWaitStrategyType.valueOf(name.get().toUpperCase()); switch (strategyType) { case BLOCKING_WAIT: return new BlockingWaitStrategy(); From d1970a3cfe95957775b7436d307107a8b1dee0e3 Mon Sep 17 00:00:00 2001 From: yuezhang Date: Tue, 25 Oct 2022 11:49:40 +0800 Subject: [PATCH 49/50] merge from master --- .../apache/hudi/common/util/queue/BoundedInMemoryExecutor.java | 1 - 1 file changed, 1 deletion(-) diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryExecutor.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryExecutor.java index f7efc3e65a4ca..0d6ff77d23d35 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryExecutor.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryExecutor.java @@ -79,7 +79,6 @@ public CompletableFuture startProducers() { return CompletableFuture.allOf(producers.stream().map(producer -> { return CompletableFuture.supplyAsync(() -> { try { - preExecuteRunnable.run(); producer.produce(queue); } catch (Throwable e) { LOG.error("error producing records", e); From 988e14a2e6ee6b2ac63573b530279ec37786041e Mon Sep 17 00:00:00 2001 From: yuezhang Date: Mon, 31 Oct 2022 17:36:56 +0800 Subject: [PATCH 50/50] address comments --- .../apache/hudi/config/HoodieWriteConfig.java | 15 ++-- .../hudi/util/QueueBasedExecutorFactory.java | 2 +- .../TestDisruptorExecutionInSpark.java | 8 +-- .../execution/TestDisruptorMessageQueue.java | 49 +++++++------ .../common/util/CustomizedThreadFactory.java | 22 +++++- .../util/queue/BoundedInMemoryExecutor.java | 3 +- .../queue/BoundedInMemoryQueueIterable.java | 5 ++ .../common/util/queue/DisruptorExecutor.java | 8 +-- .../util/queue/DisruptorMessageQueue.java | 29 +++++--- .../hudi/common/util/queue/ExecutorType.java | 2 +- .../util/queue/HoodieDaemonThreadFactory.java | 69 ------------------- .../common/util/queue/HoodieExecutorBase.java | 33 ++++----- .../common/util/queue/HoodieMessageQueue.java | 2 +- 13 files changed, 108 insertions(+), 139 deletions(-) delete mode 100644 hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieDaemonThreadFactory.java diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java index 3698678ee1558..514a4e38dc72c 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java @@ -244,15 +244,18 @@ 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 + public static final ConfigProperty WRITE_DISRUPTOR_BUFFER_SIZE = ConfigProperty .key("hoodie.write.executor.disruptor.buffer.size") - .defaultValue(1024) + .defaultValue(String.valueOf(1024)) .withDocumentation("The size of the Disruptor Executor ring buffer, must be power of 2"); public static final ConfigProperty WRITE_WAIT_STRATEGY = ConfigProperty .key("hoodie.write.executor.disruptor.wait.strategy") .defaultValue("BLOCKING_WAIT") - .withDocumentation("Strategy employed for making Disruptor Executor wait on a cursor."); + .withDocumentation("Strategy employed for making Disruptor Executor wait on a cursor. Other options are " + + "SLEEPING_WAIT, it attempts to be conservative with CPU usage by using a simple busy wait loop" + + "YIELDING_WAIT, it is designed for cases where there is the option to burn CPU cycles with the goal of improving latency" + + "BUSY_SPIN_WAIT, it can be used in low-latency systems, but puts the highest constraints on the deployment environment"); public static final ConfigProperty COMBINE_BEFORE_INSERT = ConfigProperty .key("hoodie.combine.before.insert") @@ -1069,8 +1072,8 @@ public Option getWriteExecutorWaitStrategy() { return Option.of(getString(WRITE_WAIT_STRATEGY)); } - public Option getWriteBufferSize() { - return Option.of(getInt(WRITE_BUFFER_SIZE)); + public Option getDisruptorWriteBufferSize() { + return Option.of(Integer.parseInt(getStringOrDefault(WRITE_DISRUPTOR_BUFFER_SIZE))); } public boolean shouldCombineBeforeInsert() { @@ -2377,7 +2380,7 @@ public Builder withWriteWaitStrategy(String waitStrategy) { } public Builder withWriteBufferSize(int size) { - writeConfig.setValue(WRITE_BUFFER_SIZE, String.valueOf(size)); + writeConfig.setValue(WRITE_DISRUPTOR_BUFFER_SIZE, String.valueOf(size)); return this; } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/util/QueueBasedExecutorFactory.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/util/QueueBasedExecutorFactory.java index 722d13cf604ec..ba8ddbd1ec1f1 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/util/QueueBasedExecutorFactory.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/util/QueueBasedExecutorFactory.java @@ -43,7 +43,7 @@ public static HoodieExecutor create(HoodieWriteConfig hoodieConfig, It return new BoundedInMemoryExecutor<>(hoodieConfig.getWriteBufferLimitBytes(), inputItr, consumer, transformFunction, preExecuteRunnable); case DISRUPTOR: - return new DisruptorExecutor<>(hoodieConfig.getWriteBufferSize(), inputItr, consumer, + return new DisruptorExecutor<>(hoodieConfig.getDisruptorWriteBufferSize(), inputItr, consumer, transformFunction, hoodieConfig.getWriteExecutorWaitStrategy(), preExecuteRunnable); default: throw new HoodieException("Unsupported Executor Type " + executorType); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorExecutionInSpark.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorExecutionInSpark.java index 4a61c89d9b139..2351f2bbed6df 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorExecutionInSpark.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorExecutionInSpark.java @@ -77,7 +77,7 @@ public void testExecutor() { final List consumedRecords = new ArrayList<>(); HoodieWriteConfig hoodieWriteConfig = mock(HoodieWriteConfig.class); - when(hoodieWriteConfig.getWriteBufferSize()).thenReturn(Option.of(8)); + when(hoodieWriteConfig.getDisruptorWriteBufferSize()).thenReturn(Option.of(8)); IteratorBasedQueueConsumer, Integer> consumer = new IteratorBasedQueueConsumer, Integer>() { @@ -97,7 +97,7 @@ protected Integer getResult() { DisruptorExecutor>, Integer> exec = null; try { - exec = new DisruptorExecutor(hoodieWriteConfig.getWriteBufferSize(), hoodieRecords.iterator(), consumer, + exec = new DisruptorExecutor(hoodieWriteConfig.getDisruptorWriteBufferSize(), hoodieRecords.iterator(), consumer, getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA), Option.of(WaitStrategyFactory.DEFAULT_STRATEGY), getPreExecuteRunnable()); int result = exec.execute(); // It should buffer and write 100 records @@ -125,7 +125,7 @@ public void testInterruptExecutor() { final List hoodieRecords = dataGen.generateInserts(instantTime, 100); HoodieWriteConfig hoodieWriteConfig = mock(HoodieWriteConfig.class); - when(hoodieWriteConfig.getWriteBufferSize()).thenReturn(Option.of(1024)); + when(hoodieWriteConfig.getDisruptorWriteBufferSize()).thenReturn(Option.of(1024)); IteratorBasedQueueConsumer, Integer> consumer = new IteratorBasedQueueConsumer, Integer>() { @@ -145,7 +145,7 @@ protected Integer getResult() { }; DisruptorExecutor>, Integer> - executor = new DisruptorExecutor(hoodieWriteConfig.getWriteBufferSize(), hoodieRecords.iterator(), consumer, + executor = new DisruptorExecutor(hoodieWriteConfig.getDisruptorWriteBufferSize(), hoodieRecords.iterator(), consumer, getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA), Option.of(WaitStrategyFactory.DEFAULT_STRATEGY), getPreExecuteRunnable()); try { diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorMessageQueue.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorMessageQueue.java index 4ef76c34e4f9c..d296d56440031 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorMessageQueue.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorMessageQueue.java @@ -45,6 +45,7 @@ import org.junit.jupiter.api.Timeout; import java.io.IOException; +import java.lang.reflect.Method; import java.util.ArrayList; import java.util.HashMap; import java.util.Iterator; @@ -110,7 +111,7 @@ public void testRecordReading() throws Exception { }); HoodieWriteConfig hoodieWriteConfig = mock(HoodieWriteConfig.class); - when(hoodieWriteConfig.getWriteBufferSize()).thenReturn(Option.of(16)); + when(hoodieWriteConfig.getDisruptorWriteBufferSize()).thenReturn(Option.of(16)); IteratorBasedQueueConsumer, Integer> consumer = new IteratorBasedQueueConsumer, Integer>() { @@ -138,7 +139,7 @@ protected Integer getResult() { DisruptorExecutor>, Integer> exec = null; try { - exec = new DisruptorExecutor(hoodieWriteConfig.getWriteBufferSize(), hoodieRecords.iterator(), consumer, + exec = new DisruptorExecutor(hoodieWriteConfig.getDisruptorWriteBufferSize(), hoodieRecords.iterator(), consumer, getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA), Option.of(WaitStrategyFactory.DEFAULT_STRATEGY), getPreExecuteRunnable()); int result = exec.execute(); // It should buffer and write 100 records @@ -220,26 +221,34 @@ public void run() { IntStream.range(0, numProducers).boxed().collect(Collectors.toMap(Function.identity(), x -> 0)); // setup consumer and start disruptor - queue.setHandlers(new IteratorBasedQueueConsumer, Integer>() { + IteratorBasedQueueConsumer, Integer> consumer = + new IteratorBasedQueueConsumer, Integer>() { - @Override - public void consumeOneRecord(HoodieLazyInsertIterable.HoodieInsertValueGenResult payload) { - // Read recs and ensure we have covered all producer recs. - final HoodieRecord rec = payload.record; - Pair producerPos = keyToProducerAndIndexMap.get(rec.getRecordKey()); - Integer lastSeenPos = lastSeenMap.get(producerPos.getLeft()); - countMap.put(producerPos.getLeft(), countMap.get(producerPos.getLeft()) + 1); - lastSeenMap.put(producerPos.getLeft(), lastSeenPos + 1); - // Ensure we are seeing the next record generated - assertEquals(lastSeenPos + 1, producerPos.getRight().intValue()); - } + @Override + public void consumeOneRecord(HoodieLazyInsertIterable.HoodieInsertValueGenResult payload) { + // Read recs and ensure we have covered all producer recs. + final HoodieRecord rec = payload.record; + Pair producerPos = keyToProducerAndIndexMap.get(rec.getRecordKey()); + Integer lastSeenPos = lastSeenMap.get(producerPos.getLeft()); + countMap.put(producerPos.getLeft(), countMap.get(producerPos.getLeft()) + 1); + lastSeenMap.put(producerPos.getLeft(), lastSeenPos + 1); + // Ensure we are seeing the next record generated + assertEquals(lastSeenPos + 1, producerPos.getRight().intValue()); + } - @Override - protected Integer getResult() { - return 0; - } - }); - queue.start(); + @Override + protected Integer getResult() { + return 0; + } + }; + + Method setHandlersFunc = queue.getClass().getDeclaredMethod("setHandlers", IteratorBasedQueueConsumer.class); + setHandlersFunc.setAccessible(true); + setHandlersFunc.invoke(queue, consumer); + + Method startFunc = queue.getClass().getDeclaredMethod("start"); + startFunc.setAccessible(true); + startFunc.invoke(queue); // start to produce records CompletableFuture producerFuture = CompletableFuture.allOf(producers.stream().map(producer -> { diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/CustomizedThreadFactory.java b/hudi-common/src/main/java/org/apache/hudi/common/util/CustomizedThreadFactory.java index 738be514b2cbf..a13f3a804f983 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/CustomizedThreadFactory.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/CustomizedThreadFactory.java @@ -33,6 +33,8 @@ public class CustomizedThreadFactory implements ThreadFactory { private final String threadName; private final boolean daemon; + private Runnable preExecuteRunnable; + public CustomizedThreadFactory() { this("pool-" + POOL_NUM.getAndIncrement(), false); } @@ -41,6 +43,16 @@ public CustomizedThreadFactory(String threadNamePrefix) { this(threadNamePrefix, false); } + public CustomizedThreadFactory(String threadNamePrefix, Runnable preExecuteRunnable) { + this(threadNamePrefix, false, preExecuteRunnable); + } + + public CustomizedThreadFactory(String threadNamePrefix, boolean daemon, Runnable preExecuteRunnable) { + this.threadName = threadNamePrefix + "-thread-"; + this.daemon = daemon; + this.preExecuteRunnable = preExecuteRunnable; + } + public CustomizedThreadFactory(String threadNamePrefix, boolean daemon) { this.threadName = threadNamePrefix + "-thread-"; this.daemon = daemon; @@ -48,7 +60,15 @@ public CustomizedThreadFactory(String threadNamePrefix, boolean daemon) { @Override public Thread newThread(@NotNull Runnable r) { - Thread runThread = new Thread(r); + Thread runThread = preExecuteRunnable == null ? new Thread(r) : new Thread(new Runnable() { + + @Override + public void run() { + preExecuteRunnable.run(); + r.run(); + } + }); + runThread.setDaemon(daemon); runThread.setName(threadName + threadNum.getAndIncrement()); return runThread; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryExecutor.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryExecutor.java index 0d6ff77d23d35..ce5898c7c3101 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryExecutor.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryExecutor.java @@ -110,7 +110,6 @@ protected CompletableFuture startConsumer() { return consumer.map(consumer -> { return CompletableFuture.supplyAsync(() -> { LOG.info("starting consumer thread"); - preExecuteRunnable.run(); try { E result = consumer.consume(queue); LOG.info("Queue Consumption is done; notifying producer threads"); @@ -126,7 +125,7 @@ protected CompletableFuture startConsumer() { @Override public boolean isRemaining() { - return ((BoundedInMemoryQueueIterable)queue).iterator().hasNext(); + return getQueue().iterator().hasNext(); } @Override diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryQueueIterable.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryQueueIterable.java index a0fc133391d17..47b8c81fc4600 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryQueueIterable.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryQueueIterable.java @@ -135,6 +135,7 @@ public BoundedInMemoryQueueIterable(final long memoryLimit, final Function this.iterator = new QueueIterator(); } + @Override public long size() { return this.queue.size(); } @@ -174,6 +175,7 @@ private void adjustBufferSizeIfNeeded(final O payload) throws InterruptedExcepti * * @param t Item to be queued */ + @Override public void insertRecord(I t) throws Exception { // If already closed, throw exception if (isWriteDone.get()) { @@ -203,6 +205,7 @@ private boolean expectMoreRecords() { * Reader interface but never exposed to outside world as this is a single consumer queue. Reading is done through a * singleton iterator for this queue. */ + @Override public Option readNextRecord() { if (this.isReadDone.get()) { return Option.empty(); @@ -237,6 +240,7 @@ public Option readNextRecord() { /** * Puts an empty entry to queue to denote termination. */ + @Override public void close() { // done queueing records notifying queue-reader. isWriteDone.set(true); @@ -252,6 +256,7 @@ private void throwExceptionIfFailed() { /** * API to allow producers and consumer to communicate termination due to failure. */ + @Override public void markAsFailed(Throwable e) { this.hasFailed.set(e); // release the permits so that if the queueing thread is waiting for permits then it will diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorExecutor.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorExecutor.java index 13257d884d7ae..7ea5de07c0dca 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorExecutor.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorExecutor.java @@ -18,7 +18,6 @@ package org.apache.hudi.common.util.queue; -import org.apache.hudi.common.util.Functions; import org.apache.hudi.common.util.Option; import org.apache.hudi.exception.HoodieException; @@ -48,11 +47,6 @@ public DisruptorExecutor(final Option bufferSize, final Iterator inp this(bufferSize, new IteratorBasedQueueProducer<>(inputItr), Option.of(consumer), transformFunction, waitStrategy, preExecuteRunnable); } - public DisruptorExecutor(final Option bufferSize, HoodieProducer producer, - Option> consumer, final Function transformFunction) { - this(bufferSize, producer, consumer, transformFunction, Option.of(WaitStrategyFactory.DEFAULT_STRATEGY), Functions.noop()); - } - public DisruptorExecutor(final Option bufferSize, HoodieProducer producer, Option> consumer, final Function transformFunction, Option waitStrategy, Runnable preExecuteRunnable) { this(bufferSize, Collections.singletonList(producer), consumer, transformFunction, waitStrategy, preExecuteRunnable); @@ -92,8 +86,8 @@ protected void setup() { @Override protected void postAction() { try { - queue.close(); super.close(); + queue.close(); } catch (IOException e) { throw new HoodieIOException("Catch IOException while closing DisruptorMessageQueue", e); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageQueue.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageQueue.java index 975bfa8dbbcf5..eccd881af1431 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageQueue.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageQueue.java @@ -25,10 +25,13 @@ import com.lmax.disruptor.WaitStrategy; import com.lmax.disruptor.dsl.Disruptor; import com.lmax.disruptor.dsl.ProducerType; +import org.apache.hudi.common.util.CustomizedThreadFactory; import org.apache.hudi.common.util.Option; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; import java.util.function.Function; public class DisruptorMessageQueue implements HoodieMessageQueue { @@ -38,10 +41,13 @@ public class DisruptorMessageQueue implements HoodieMessageQueue { private final Disruptor queue; private final Function transformFunction; private final RingBuffer ringBuffer; + private final Lock closeLocker = new ReentrantLock(); + + private boolean isDisruptorClosed = false; public DisruptorMessageQueue(Option bufferSize, Function transformFunction, Option waitStrategyName, int totalProducers, Runnable preExecuteRunnable) { WaitStrategy waitStrategy = WaitStrategyFactory.build(waitStrategyName); - HoodieDaemonThreadFactory threadFactory = new HoodieDaemonThreadFactory(preExecuteRunnable); + CustomizedThreadFactory threadFactory = new CustomizedThreadFactory("disruptor", true, preExecuteRunnable); this.queue = new Disruptor<>(new HoodieDisruptorEventFactory(), bufferSize.get(), threadFactory, totalProducers > 1 ? ProducerType.MULTI : ProducerType.SINGLE, waitStrategy); this.ringBuffer = queue.getRingBuffer(); @@ -50,7 +56,7 @@ public DisruptorMessageQueue(Option bufferSize, Function transfor @Override public long size() { - return queue.getBufferSize(); + return ringBuffer.getBufferSize() - ringBuffer.remainingCapacity(); } @Override @@ -71,16 +77,21 @@ public void markAsFailed(Throwable e) { } @Override - public void close() { - // Waits until all events currently in the disruptor have been processed by all event processors - queue.shutdown(); - } - public boolean isEmpty() { return ringBuffer.getBufferSize() == ringBuffer.remainingCapacity(); } - public void setHandlers(IteratorBasedQueueConsumer consumer) { + @Override + public void close() { + closeLocker.lock(); + if (!isDisruptorClosed) { + queue.shutdown(); + isDisruptorClosed = true; + } + closeLocker.unlock(); + } + + protected void setHandlers(IteratorBasedQueueConsumer consumer) { queue.handleEventsWith(new EventHandler() { @Override @@ -90,7 +101,7 @@ public void onEvent(HoodieDisruptorEvent event, long sequence, boolean endOfBatc }); } - public void start() { + protected void start() { queue.start(); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/ExecutorType.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/ExecutorType.java index 08d37ad1d7ecf..05ecb1746c28d 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/ExecutorType.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/ExecutorType.java @@ -41,7 +41,7 @@ public enum ExecutorType { DISRUPTOR; public static List getNames() { - List names = new ArrayList<>(KeyGeneratorType.values().length); + List names = new ArrayList<>(ExecutorType.values().length); Arrays.stream(KeyGeneratorType.values()) .forEach(x -> names.add(x.name())); return names; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieDaemonThreadFactory.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieDaemonThreadFactory.java deleted file mode 100644 index a2625b5a03db3..0000000000000 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieDaemonThreadFactory.java +++ /dev/null @@ -1,69 +0,0 @@ -/* - * 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.apache.hudi.common.util.Functions; - -import java.util.concurrent.ThreadFactory; -import java.util.concurrent.atomic.AtomicInteger; - -/** - * Access to a ThreadFactory instance. - * 1. All threads are created with setDaemon(true). - * 2. All threads execute preExecuteRunnable func once. - */ -public class HoodieDaemonThreadFactory implements ThreadFactory { - - private final Runnable preExecuteRunnable; - private final AtomicInteger threadsNum = new AtomicInteger(); - private final String namePattern; - private static final String BASE_NAME = "Hoodie-daemon-thread"; - - public HoodieDaemonThreadFactory() { - this(BASE_NAME, Functions.noop()); - } - - public HoodieDaemonThreadFactory(String threadNamePrefix) { - this(threadNamePrefix, Functions.noop()); - } - - public HoodieDaemonThreadFactory(Runnable preExecuteRunnable) { - this(BASE_NAME, preExecuteRunnable); - } - - public HoodieDaemonThreadFactory(String threadNamePrefix, Runnable preExecuteRunnable) { - this.preExecuteRunnable = preExecuteRunnable; - this.namePattern = threadNamePrefix + "-%d"; - } - - @Override - public Thread newThread(Runnable r) { - Thread t = new Thread(new Runnable() { - - @Override - public void run() { - preExecuteRunnable.run(); - r.run(); - } - }, String.format(namePattern, threadsNum.addAndGet(1))); - - t.setDaemon(true); - return t; - } -} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieExecutorBase.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieExecutorBase.java index e123924ec8c68..8dc07e35e4901 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieExecutorBase.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieExecutorBase.java @@ -41,17 +41,17 @@ public abstract class HoodieExecutorBase implements HoodieExecutor> producers; + protected final List> producers; // Consumer - public final Option> consumer; + protected final Option> consumer; // pre-execute function to implement environment specific behavior before executors (producers/consumer) run - public final Runnable preExecuteRunnable; + protected final Runnable preExecuteRunnable; - public CompletableFuture producerFuture; + CompletableFuture producerFuture; public HoodieExecutorBase(List> producers, Option> consumer, Runnable preExecuteRunnable) { @@ -59,9 +59,9 @@ public HoodieExecutorBase(List> producers, Option> producers, Option extends Closeable { /** - * Get the size of inner message queue. + * Returns the number of elements in this queue. */ long size();