You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hudi.apache.org by GitBox <gi...@apache.org> on 2022/09/30 01:23:07 UTC

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

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


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

Review Comment:
   Same as above



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

Review Comment:
    - Let's convert this to an interface
    - Please add a java-doc



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

Review Comment:
   Can you help me understand what kind of prologues we're planning to execute here?



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

Review Comment:
   If we're planning to have a custom factory it's a good idea to add custom name to the threads it produces (for them to be more easily identifiable)



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

Review Comment:
   Let's make it final



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

Review Comment:
   Would suggest to sub-scope this to (for ex, `hoodie.write.executor.disruptor.buffer.size`)



##########
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<I> implements HoodieProducer<I> {

Review Comment:
   We should make both producers/consumers invariant of the Queue impl, in other words we should keep implementation details (BoundedLockingQueue, Disruptor, etc) abstracted away from them (there're simply no reason for them to be aware what's the implementation details of the queue).
   
    - We should keep only `HoodieQueueProducer` (removing specific ones)
    - Same for Consumers



##########
hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorExecutor.java:
##########
@@ -0,0 +1,168 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.common.util.queue;
+
+import com.lmax.disruptor.dsl.Disruptor;
+import org.apache.hudi.common.util.CustomizedThreadFactory;
+import org.apache.hudi.common.util.Functions;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.exception.HoodieException;
+
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+public class DisruptorExecutor<I, O, E> extends HoodieExecutor<I, O, E> {
+
+  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<I, O> queue;
+  // Producers
+  private final List<DisruptorBasedProducer<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;
+
+  public DisruptorExecutor(final int bufferSize, final Iterator<I> inputItr,
+                           BoundedInMemoryQueueConsumer<O, E> consumer, Function<I, O> transformFunction, String waitStrategy, Runnable preExecuteRunnable) {
+    this(bufferSize, new IteratorBasedDisruptorProducer<>(inputItr), Option.of(consumer), transformFunction, waitStrategy, preExecuteRunnable);
+  }
+
+  public DisruptorExecutor(final int bufferSize, DisruptorBasedProducer<I> producer,
+                           Option<BoundedInMemoryQueueConsumer<O, E>> consumer, final Function<I, O> transformFunction) {
+    this(bufferSize, producer, consumer, transformFunction, WaitStrategyFactory.DEFAULT_STRATEGY, Functions.noop());
+  }
+
+  public DisruptorExecutor(final int bufferSize, DisruptorBasedProducer<I> producer,
+                           Option<BoundedInMemoryQueueConsumer<O, E>> consumer, final Function<I, O> transformFunction, String waitStrategy, Runnable preExecuteRunnable) {
+    this(bufferSize, Collections.singletonList(producer), consumer, transformFunction, waitStrategy, preExecuteRunnable);
+  }
+
+  public DisruptorExecutor(final int bufferSize, List<DisruptorBasedProducer<I>> producers,
+                           Option<BoundedInMemoryQueueConsumer<O, E>> consumer, final Function<I, O> transformFunction,
+                           final String waitStrategy, Runnable preExecuteRunnable) {
+    this.producers = producers;
+    this.consumer = consumer;
+    this.preExecuteRunnable = preExecuteRunnable;
+    // Ensure fixed thread for each producer thread
+    this.producerExecutorService = Executors.newFixedThreadPool(producers.size(), new CustomizedThreadFactory("producer"));
+    // Ensure single thread for consumer
+    this.consumerExecutorService = Executors.newSingleThreadExecutor(new CustomizedThreadFactory("consumer"));
+    this.queue = new DisruptorMessageQueue<>(bufferSize, transformFunction, waitStrategy, producers.size(), preExecuteRunnable);
+  }
+
+  /**
+   * Start all Producers.
+   */
+  public ExecutorCompletionService<Boolean> startProducers() {
+    final ExecutorCompletionService<Boolean> completionService =
+        new ExecutorCompletionService<Boolean>(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 {
+      assert consumer.isPresent();

Review Comment:
   Please use `checkState` util for that



##########
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<I, O> {
+
+  /**
+   * Consumer One record.
+   */
+  protected abstract void consumeOneRecord(I record);
+
+  /**
+   * Notifies implementation that we have exhausted consuming records from queue.
+   */
+  protected abstract void finish();
+
+  /**
+   * Return result of consuming records so far.
+   */
+  protected abstract O getResult();

Review Comment:
   Do we need this method in the interface?



##########
hudi-common/src/main/java/org/apache/hudi/common/util/queue/WaitStrategyFactory.java:
##########
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.common.util.queue;
+
+import static org.apache.hudi.common.util.queue.DisruptorWaitStrategyType.BLOCKINGWAITSTRATEGY;
+
+import com.lmax.disruptor.BlockingWaitStrategy;
+import com.lmax.disruptor.BusySpinWaitStrategy;
+import com.lmax.disruptor.SleepingWaitStrategy;
+import com.lmax.disruptor.WaitStrategy;
+import com.lmax.disruptor.YieldingWaitStrategy;
+import org.apache.hudi.exception.HoodieException;
+
+public class WaitStrategyFactory {
+
+  public static final String DEFAULT_STRATEGY = BLOCKINGWAITSTRATEGY.name();
+
+  /**
+   * Build WaitStrategy for disruptor
+   */
+  public static WaitStrategy build(String name) {
+
+    DisruptorWaitStrategyType strategyType = DisruptorWaitStrategyType.valueOf(name.toUpperCase());
+    switch (strategyType) {
+      case BLOCKINGWAITSTRATEGY:

Review Comment:
   Let's use snake-case for these: `BLOCKING_WAIT_STRATEGY`



##########
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<I, O> {

Review Comment:
   Same comments as above (please ditto for all interfaces)



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

Review Comment:
   Please make sure you're using Javax annotations (not Jetbrains's ones)



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

Review Comment:
   What do we need to have this wrapper for?



##########
hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorExecutor.java:
##########
@@ -0,0 +1,168 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.common.util.queue;
+
+import com.lmax.disruptor.dsl.Disruptor;
+import org.apache.hudi.common.util.CustomizedThreadFactory;
+import org.apache.hudi.common.util.Functions;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.exception.HoodieException;
+
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+public class DisruptorExecutor<I, O, E> extends HoodieExecutor<I, O, E> {
+
+  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<I, O> queue;
+  // Producers
+  private final List<DisruptorBasedProducer<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;
+
+  public DisruptorExecutor(final int bufferSize, final Iterator<I> inputItr,
+                           BoundedInMemoryQueueConsumer<O, E> consumer, Function<I, O> transformFunction, String waitStrategy, Runnable preExecuteRunnable) {
+    this(bufferSize, new IteratorBasedDisruptorProducer<>(inputItr), Option.of(consumer), transformFunction, waitStrategy, preExecuteRunnable);
+  }
+
+  public DisruptorExecutor(final int bufferSize, DisruptorBasedProducer<I> producer,
+                           Option<BoundedInMemoryQueueConsumer<O, E>> consumer, final Function<I, O> transformFunction) {
+    this(bufferSize, producer, consumer, transformFunction, WaitStrategyFactory.DEFAULT_STRATEGY, Functions.noop());
+  }
+
+  public DisruptorExecutor(final int bufferSize, DisruptorBasedProducer<I> producer,
+                           Option<BoundedInMemoryQueueConsumer<O, E>> consumer, final Function<I, O> transformFunction, String waitStrategy, Runnable preExecuteRunnable) {
+    this(bufferSize, Collections.singletonList(producer), consumer, transformFunction, waitStrategy, preExecuteRunnable);
+  }
+
+  public DisruptorExecutor(final int bufferSize, List<DisruptorBasedProducer<I>> producers,
+                           Option<BoundedInMemoryQueueConsumer<O, E>> consumer, final Function<I, O> transformFunction,
+                           final String waitStrategy, Runnable preExecuteRunnable) {
+    this.producers = producers;
+    this.consumer = consumer;
+    this.preExecuteRunnable = preExecuteRunnable;
+    // Ensure fixed thread for each producer thread
+    this.producerExecutorService = Executors.newFixedThreadPool(producers.size(), new CustomizedThreadFactory("producer"));
+    // Ensure single thread for consumer
+    this.consumerExecutorService = Executors.newSingleThreadExecutor(new CustomizedThreadFactory("consumer"));

Review Comment:
   Since Disruptor spawns its own threads, we can just stub this one out (no need to allocate any threads for it)



##########
hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorExecutor.java:
##########
@@ -0,0 +1,168 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.common.util.queue;
+
+import com.lmax.disruptor.dsl.Disruptor;
+import org.apache.hudi.common.util.CustomizedThreadFactory;
+import org.apache.hudi.common.util.Functions;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.exception.HoodieException;
+
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+public class DisruptorExecutor<I, O, E> extends HoodieExecutor<I, O, E> {
+
+  private static final Logger LOG = LogManager.getLogger(DisruptorExecutor.class);
+
+  // Executor service used for launching write thread.
+  private final ExecutorService producerExecutorService;

Review Comment:
   Let's create common base class `HoodieExecutorBase` abstracting common components for both executors (executor services, producers, consumers, etc)



##########
hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorExecutor.java:
##########
@@ -0,0 +1,168 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.common.util.queue;
+
+import com.lmax.disruptor.dsl.Disruptor;
+import org.apache.hudi.common.util.CustomizedThreadFactory;
+import org.apache.hudi.common.util.Functions;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.exception.HoodieException;
+
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+public class DisruptorExecutor<I, O, E> extends HoodieExecutor<I, O, E> {
+
+  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<I, O> queue;
+  // Producers
+  private final List<DisruptorBasedProducer<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;
+
+  public DisruptorExecutor(final int bufferSize, final Iterator<I> inputItr,
+                           BoundedInMemoryQueueConsumer<O, E> consumer, Function<I, O> transformFunction, String waitStrategy, Runnable preExecuteRunnable) {
+    this(bufferSize, new IteratorBasedDisruptorProducer<>(inputItr), Option.of(consumer), transformFunction, waitStrategy, preExecuteRunnable);
+  }
+
+  public DisruptorExecutor(final int bufferSize, DisruptorBasedProducer<I> producer,
+                           Option<BoundedInMemoryQueueConsumer<O, E>> consumer, final Function<I, O> transformFunction) {
+    this(bufferSize, producer, consumer, transformFunction, WaitStrategyFactory.DEFAULT_STRATEGY, Functions.noop());
+  }
+
+  public DisruptorExecutor(final int bufferSize, DisruptorBasedProducer<I> producer,
+                           Option<BoundedInMemoryQueueConsumer<O, E>> consumer, final Function<I, O> transformFunction, String waitStrategy, Runnable preExecuteRunnable) {
+    this(bufferSize, Collections.singletonList(producer), consumer, transformFunction, waitStrategy, preExecuteRunnable);
+  }
+
+  public DisruptorExecutor(final int bufferSize, List<DisruptorBasedProducer<I>> producers,
+                           Option<BoundedInMemoryQueueConsumer<O, E>> consumer, final Function<I, O> transformFunction,
+                           final String waitStrategy, Runnable preExecuteRunnable) {
+    this.producers = producers;
+    this.consumer = consumer;
+    this.preExecuteRunnable = preExecuteRunnable;
+    // Ensure fixed thread for each producer thread
+    this.producerExecutorService = Executors.newFixedThreadPool(producers.size(), new CustomizedThreadFactory("producer"));
+    // Ensure single thread for consumer
+    this.consumerExecutorService = Executors.newSingleThreadExecutor(new CustomizedThreadFactory("consumer"));
+    this.queue = new DisruptorMessageQueue<>(bufferSize, transformFunction, waitStrategy, producers.size(), preExecuteRunnable);
+  }
+
+  /**
+   * Start all Producers.
+   */
+  public ExecutorCompletionService<Boolean> startProducers() {
+    final ExecutorCompletionService<Boolean> completionService =
+        new ExecutorCompletionService<Boolean>(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 {
+      assert consumer.isPresent();
+      setupConsumer();
+      ExecutorCompletionService<Boolean> pool = startProducers();
+
+      waitForProducersFinished(pool);
+      queue.getInnerQueue().shutdown();
+      consumer.get().finish();
+
+      return consumer.get().getResult();
+    } catch (InterruptedException ie) {
+      shutdownNow();
+      Thread.currentThread().interrupt();
+      throw new HoodieException(ie);
+    } catch (Exception e) {
+      throw new HoodieException(e);
+    }
+  }
+
+  private void waitForProducersFinished(ExecutorCompletionService<Boolean> pool) throws InterruptedException, ExecutionException {
+    for (int i = 0; i < producers.size(); i++) {
+      pool.take().get();
+    }
+  }
+
+  /**
+   * Start only consumer.
+   */
+  private void setupConsumer() {
+    DisruptorMessageHandler<O, E> handler = new DisruptorMessageHandler<>(consumer.get());
+
+    Disruptor<HoodieDisruptorEvent<O>> innerQueue = queue.getInnerQueue();
+    innerQueue.handleEventsWith(handler);
+    innerQueue.start();
+  }
+
+  @Override
+  public boolean isRemaining() {
+    return !queue.isEmpty();
+  }
+
+  @Override
+  public void shutdownNow() {
+    producerExecutorService.shutdownNow();
+    consumerExecutorService.shutdownNow();
+    queue.close();
+  }
+
+  @Override
+  public DisruptorMessageQueue<I, O> getQueue() {
+    return queue;
+  }
+
+  @Override
+  public boolean awaitTermination() {

Review Comment:
   Please check my comment above: both BoundedInMemory and Disruptor Executors could share these sequences (for managing lifecycles of executor services)



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

Review Comment:
   We should not be exposing this method



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

Review Comment:
   Let's also add java-docs for the methods



##########
hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorExecutor.java:
##########
@@ -0,0 +1,163 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.common.util.queue;
+
+import com.lmax.disruptor.dsl.Disruptor;
+import org.apache.hudi.common.util.CustomizedThreadFactory;
+import org.apache.hudi.common.util.Functions;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.exception.HoodieException;
+
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+public class DisruptorExecutor<I, O, E> extends HoodieExecutor<I, O, E> {
+
+  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<I, O> queue;
+  // Producers
+  private final List<DisruptorBasedProducer<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;
+
+  public DisruptorExecutor(final int bufferSize, final Iterator<I> inputItr,
+                           BoundedInMemoryQueueConsumer<O, E> consumer, Function<I, O> transformFunction, String waitStrategy, Runnable preExecuteRunnable) {
+    this(bufferSize, new IteratorBasedDisruptorProducer<>(inputItr), Option.of(consumer), transformFunction, waitStrategy, preExecuteRunnable);
+  }
+
+  public DisruptorExecutor(final int bufferSize, DisruptorBasedProducer<I> producer,
+                           Option<BoundedInMemoryQueueConsumer<O, E>> consumer, final Function<I, O> transformFunction) {
+    this(bufferSize, producer, consumer, transformFunction, WaitStrategyFactory.DEFAULT_STRATEGY, Functions.noop());
+  }
+
+  public DisruptorExecutor(final int bufferSize, DisruptorBasedProducer<I> producer,
+                           Option<BoundedInMemoryQueueConsumer<O, E>> consumer, final Function<I, O> transformFunction, String waitStrategy, Runnable preExecuteRunnable) {
+    this(bufferSize, Collections.singletonList(producer), consumer, transformFunction, waitStrategy, preExecuteRunnable);
+  }
+
+  public DisruptorExecutor(final int bufferSize, List<DisruptorBasedProducer<I>> producers,
+                           Option<BoundedInMemoryQueueConsumer<O, E>> consumer, final Function<I, O> transformFunction,
+                           final String waitStrategy, Runnable preExecuteRunnable) {
+    this.producers = producers;
+    this.consumer = consumer;
+    this.preExecuteRunnable = preExecuteRunnable;
+    // Ensure fixed thread for each producer thread
+    this.producerExecutorService = Executors.newFixedThreadPool(producers.size(), new CustomizedThreadFactory("producer"));
+    // Ensure single thread for consumer
+    this.consumerExecutorService = Executors.newSingleThreadExecutor(new CustomizedThreadFactory("consumer"));
+    this.queue = new DisruptorMessageQueue<>(bufferSize, transformFunction, waitStrategy, producers.size(), preExecuteRunnable);
+  }
+
+  /**
+   * Start all Producers.
+   */
+  public ExecutorCompletionService<Boolean> startProducers() {
+    final ExecutorCompletionService<Boolean> completionService =
+        new ExecutorCompletionService<Boolean>(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() {

Review Comment:
   Strongly agree w/ @YuweiXiao here. Let's try our best to unify these as much as possible.
   
   For the setup sequence we can create empty `setup` method in the base which will be no-op for Bounded



##########
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<I, O> {

Review Comment:
   This API seems incomplete: we do have methods to insert into the queue, but we don't have a method to take from the queue



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

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

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