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


##########
hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorExecutionInSpark.java:
##########
@@ -0,0 +1,159 @@
+/*
+ * 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.IteratorBasedQueueConsumer;
+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;
+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.util.ArrayList;
+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<HoodieRecord> hoodieRecords = 
dataGen.generateInserts(instantTime, 128);
+    final List<HoodieRecord> consumedRecords = new ArrayList<>();
+
+    HoodieWriteConfig hoodieWriteConfig = mock(HoodieWriteConfig.class);
+    when(hoodieWriteConfig.getWriteBufferSize()).thenReturn(Option.of(8));
+    
IteratorBasedQueueConsumer<HoodieLazyInsertIterable.HoodieInsertValueGenResult<HoodieRecord>,
 Integer> consumer =
+        new 
IteratorBasedQueueConsumer<HoodieLazyInsertIterable.HoodieInsertValueGenResult<HoodieRecord>,
 Integer>() {
+
+          private int count = 0;
+
+          @Override
+          public void 
consumeOneRecord(HoodieLazyInsertIterable.HoodieInsertValueGenResult<HoodieRecord>
 record) {
+            consumedRecords.add(record.record);
+            count++;
+          }
+
+          @Override
+          protected Integer getResult() {
+            return count;
+          }
+        };
+    DisruptorExecutor<HoodieRecord, Tuple2<HoodieRecord, 
Option<IndexedRecord>>, Integer> exec = null;
+
+    try {
+      exec = new DisruptorExecutor(hoodieWriteConfig.getWriteBufferSize(), 
hoodieRecords.iterator(), consumer,
+          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());

Review Comment:
   changed!



##########
hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryExecutor.java:
##########
@@ -18,96 +18,77 @@
 
 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;
 import org.apache.hudi.common.util.SizeEstimator;
 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.CompletableFuture;
 import java.util.concurrent.CountDownLatch;
 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;
 
 /**
- * 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<I, O, E> {
+public class BoundedInMemoryExecutor<I, O, E> extends HoodieExecutorBase<I, O, 
E> {
 
   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<I, O> queue;
-  // Producers
-  private final List<BoundedInMemoryQueueProducer<I>> producers;
-  // Consumer
-  private final Option<BoundedInMemoryQueueConsumer<O, E>> consumer;
-  // pre-execute function to implement environment specific behavior before 
executors (producers/consumer) run
-  private final Runnable preExecuteRunnable;
+  private final HoodieMessageQueue<I, O> queue;
 
   public BoundedInMemoryExecutor(final long bufferLimitInBytes, final 
Iterator<I> inputItr,
-                                 BoundedInMemoryQueueConsumer<O, E> consumer, 
Function<I, O> transformFunction, Runnable preExecuteRunnable) {
+                                 IteratorBasedQueueConsumer<O, E> consumer, 
Function<I, O> transformFunction, Runnable preExecuteRunnable) {
     this(bufferLimitInBytes, new IteratorBasedQueueProducer<>(inputItr), 
Option.of(consumer), transformFunction, preExecuteRunnable);
   }
 
-  public BoundedInMemoryExecutor(final long bufferLimitInBytes, 
BoundedInMemoryQueueProducer<I> producer,
-                                 Option<BoundedInMemoryQueueConsumer<O, E>> 
consumer, final Function<I, O> transformFunction) {
+  public BoundedInMemoryExecutor(final long bufferLimitInBytes, 
HoodieProducer<I> producer,
+                                 Option<IteratorBasedQueueConsumer<O, E>> 
consumer, final Function<I, O> transformFunction) {
     this(bufferLimitInBytes, producer, consumer, transformFunction, 
Functions.noop());
   }
 
-  public BoundedInMemoryExecutor(final long bufferLimitInBytes, 
BoundedInMemoryQueueProducer<I> producer,
-                                 Option<BoundedInMemoryQueueConsumer<O, E>> 
consumer, final Function<I, O> transformFunction, Runnable preExecuteRunnable) {
+  public BoundedInMemoryExecutor(final long bufferLimitInBytes, 
HoodieProducer<I> producer,
+                                 Option<IteratorBasedQueueConsumer<O, E>> 
consumer, final Function<I, O> transformFunction, Runnable preExecuteRunnable) {
     this(bufferLimitInBytes, Collections.singletonList(producer), consumer, 
transformFunction, new DefaultSizeEstimator<>(), preExecuteRunnable);
   }
 
-  public BoundedInMemoryExecutor(final long bufferLimitInBytes, 
List<BoundedInMemoryQueueProducer<I>> producers,
-                                 Option<BoundedInMemoryQueueConsumer<O, E>> 
consumer, final Function<I, O> transformFunction,
+  public BoundedInMemoryExecutor(final long bufferLimitInBytes, 
List<HoodieProducer<I>> producers,
+                                 Option<IteratorBasedQueueConsumer<O, E>> 
consumer, final Function<I, O> transformFunction,
                                  final SizeEstimator<O> 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"));
+    super(producers, consumer, preExecuteRunnable);
     this.queue = new BoundedInMemoryQueue<>(bufferLimitInBytes, 
transformFunction, sizeEstimator);
   }
 
   /**
-   * Start all Producers.
+   * Start all producers at once.
    */
-  public ExecutorCompletionService<Boolean> startProducers() {
+  public List<Future<Boolean>> startProducers() {

Review Comment:
   All changed.



##########
hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieExecutorBase.java:
##########
@@ -0,0 +1,119 @@
+/*
+ * 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.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 a single consumer.
+ * Also HoodieExecutorBase control the lifecycle of producerExecutorService 
and consumerExecutorService.
+ */
+public abstract class HoodieExecutorBase<I, O, E> implements HoodieExecutor<I, 
O, E> {
+
+  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.
+  public final ExecutorService producerExecutorService;
+  // Executor service used for launching read thread.
+  public final ExecutorService consumerExecutorService;
+  // Producers
+  public final List<HoodieProducer<I>> producers;
+  // Consumer
+  public final Option<IteratorBasedQueueConsumer<O, E>> consumer;
+  // pre-execute function to implement environment specific behavior before 
executors (producers/consumer) run
+  public final Runnable preExecuteRunnable;
+
+  public List<Future<Boolean>> producerTasks;
+
+  public HoodieExecutorBase(List<HoodieProducer<I>> producers, 
Option<IteratorBasedQueueConsumer<O, E>> 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 
HoodieDaemonThreadFactory("producer", preExecuteRunnable));

Review Comment:
   Changed.



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

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to