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


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

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



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

Review Comment:
   java docs



##########
hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/execution/benchmark/BoundInMemoryExecutorBenchmark.scala:
##########
@@ -0,0 +1,133 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.benchmark
+
+import org.apache.hadoop.fs.Path
+import org.apache.hudi.HoodieSparkUtils
+import org.apache.spark.SparkConf
+import org.apache.spark.hudi.benchmark.{HoodieBenchmark, HoodieBenchmarkBase}
+import org.apache.spark.sql.hudi.HoodieSparkSessionExtension
+import org.apache.spark.sql.types._
+import org.apache.spark.sql.{DataFrame, RowFactory, SaveMode, SparkSession}
+
+import scala.util.Random
+
+object BoundInMemoryExecutorBenchmark extends HoodieBenchmarkBase {
+
+  protected val spark: SparkSession = getSparkSession
+
+  val recordNumber = 1000000
+
+  def getSparkSession: SparkSession = SparkSession.builder()
+    .master("local[*]")
+    .appName(this.getClass.getCanonicalName)
+    .withExtensions(new HoodieSparkSessionExtension)
+    .config("spark.serializer", "org.apache.spark.serializer.KryoSerializer")
+    .config("spark.sql.session.timeZone", "CTT")
+    .config(sparkConf())
+    .getOrCreate()
+
+  def sparkConf(): SparkConf = {
+    val sparkConf = new SparkConf()
+    if (HoodieSparkUtils.gteqSpark3_2) {
+      sparkConf.set("spark.sql.catalog.spark_catalog",
+        "org.apache.spark.sql.hudi.catalog.HoodieCatalog")
+    }
+    sparkConf
+  }
+
+  private def createDataFrame(number: Int): DataFrame = {
+    val schema = new StructType()
+      .add("c1", IntegerType)
+      .add("c2", StringType)
+
+    val rdd = spark.sparkContext.parallelize(0 to number, 2).map { item =>
+      val c1 = Integer.valueOf(item)
+      val c2 = s"abc"
+      RowFactory.create(c1, c2)
+    }
+    spark.createDataFrame(rdd, schema)
+  }
+
+  /**
+   * Intel(R) Xeon(R) Platinum 8259CL CPU @ 2.50GHz
+   * COW Ingestion:                            Best Time(ms)   Avg Time(ms)   
Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
+   * 
------------------------------------------------------------------------------------------------------------------------
+   * BoundInMemory Executor                             5557           5607    
      70          0.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)

Review Comment:
   curious to know if you tried other Wait strategies



##########
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:
   also, why return true directly. should we check for producerExecutorService 
termination and consumerExecutorService termination. similar logic as followed 
in BoundedInMemoryExecutor



##########
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> {

Review Comment:
   java docs please



##########
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;

Review Comment:
   final



##########
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);

Review Comment:
   `consumeRecord` should suffice IMO. why name it `consumeOneRecord`



##########
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
+      .key("hoodie.write.wait.strategy")
+      .defaultValue("BlockingWaitStrategy")

Review Comment:
   again, may be an enum for this too. and you can remove "Strategy" suffix. 



##########
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,

Review Comment:
   oh, we do have an enum defined. may be we can remote "EXECUTOR" suffix. its 
repetitive



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

Review Comment:
   left comment elsewhere. we should move the produce method to this interface. 



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/SparkLazyInsertIterable.java:
##########
@@ -77,16 +80,28 @@ public SparkLazyInsertIterable(Iterator<HoodieRecord<T>> 
recordItr,
   @Override
   protected List<WriteStatus> computeNext() {
     // Executor service used for launching writer thread.
-    BoundedInMemoryExecutor<HoodieRecord<T>, 
HoodieInsertValueGenResult<HoodieRecord>, List<WriteStatus>> 
bufferedIteratorExecutor =
-        null;
+    HoodieExecutor<?, ?, List<WriteStatus>> bufferedIteratorExecutor = null;
     try {
       Schema schema = new Schema.Parser().parse(hoodieConfig.getSchema());
       if (useWriterSchema) {
         schema = HoodieAvroUtils.addMetadataFields(schema);
       }
-      bufferedIteratorExecutor =
-          new 
BoundedInMemoryExecutor<>(hoodieConfig.getWriteBufferLimitBytes(), inputItr, 
getInsertHandler(),
+
+      ExecutorType executorType = hoodieConfig.getExecutorType();
+
+      switch (executorType) {
+        case BOUNDED_IN_MEMORY_EXECUTOR:

Review Comment:
   I have just started reviewing. not sure if we have similar code elsewhere. 
but can we move this to a util method so that code here is leaner and can be 
reused at other places. 
   
   ```
   bufferedIteratorExecutor = 
MessageQueueUtil/MessageQueueFactory.getMessageQueueExecutor(executorType size, 
itr, consumer, transformFn, preExecutorRunnable).  
   ```



##########
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,

Review Comment:
   can remove "STRATEGY" suffix. its repetitive



##########
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);

Review Comment:
   we can do ternary operator. 
   ```
   this.queue = new Disruptor<>(HoodieDisruptorEvent::new, bufferSize, 
threadFactory, producerNumer >1 ? ProducerType.MULTI : ProducerType.SINGLE, 
waitStrategy);
   ```



##########
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();

Review Comment:
   so, is this expected to be executed by every spark task i.e. for every 
producer? 



##########
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 {

Review Comment:
   java docs please



##########
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> {
+
+  public abstract long size();
+
+  public abstract void insertRecord(I t) throws Exception;
+
+  public abstract void close();

Review Comment:
   instead of adding this method explicitly, you can extend from Closeable 
interface. 



##########
hudi-common/src/main/java/org/apache/hudi/common/util/queue/FunctionBasedDisruptorProducer.java:
##########
@@ -0,0 +1,42 @@
+/*
+ * 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<I> extends 
DisruptorBasedProducer<I> {

Review Comment:
   can we add java docs to all new classes, interfaces, abstract classes please.



##########
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();

Review Comment:
   queue.close()



##########
pom.xml:
##########
@@ -1069,6 +1070,12 @@
         <version>${zk-curator.version}</version>
       </dependency>
 
+      <dependency>
+        <groupId>com.lmax</groupId>
+        <artifactId>disruptor</artifactId>
+        <version>${disruptor.version}</version>
+      </dependency>

Review Comment:
   only hudi-common needs this right ? why add it to root pom ? can you help me 
understand. 



##########
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) {

Review Comment:
   `producerNumber` -> `producerCount` or `totalProducers`



##########
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;

Review Comment:
   can we throw HoodieException here



##########
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<O> implements 
EventFactory<HoodieDisruptorEvent<O>> {

Review Comment:
   java docs



##########
hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/execution/benchmark/BoundInMemoryExecutorBenchmark.scala:
##########
@@ -0,0 +1,133 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.benchmark
+
+import org.apache.hadoop.fs.Path
+import org.apache.hudi.HoodieSparkUtils
+import org.apache.spark.SparkConf
+import org.apache.spark.hudi.benchmark.{HoodieBenchmark, HoodieBenchmarkBase}
+import org.apache.spark.sql.hudi.HoodieSparkSessionExtension
+import org.apache.spark.sql.types._
+import org.apache.spark.sql.{DataFrame, RowFactory, SaveMode, SparkSession}
+
+import scala.util.Random
+
+object BoundInMemoryExecutorBenchmark extends HoodieBenchmarkBase {
+
+  protected val spark: SparkSession = getSparkSession
+
+  val recordNumber = 1000000
+
+  def getSparkSession: SparkSession = SparkSession.builder()
+    .master("local[*]")
+    .appName(this.getClass.getCanonicalName)
+    .withExtensions(new HoodieSparkSessionExtension)
+    .config("spark.serializer", "org.apache.spark.serializer.KryoSerializer")
+    .config("spark.sql.session.timeZone", "CTT")
+    .config(sparkConf())
+    .getOrCreate()
+
+  def sparkConf(): SparkConf = {
+    val sparkConf = new SparkConf()
+    if (HoodieSparkUtils.gteqSpark3_2) {
+      sparkConf.set("spark.sql.catalog.spark_catalog",
+        "org.apache.spark.sql.hudi.catalog.HoodieCatalog")
+    }
+    sparkConf
+  }
+
+  private def createDataFrame(number: Int): DataFrame = {
+    val schema = new StructType()
+      .add("c1", IntegerType)
+      .add("c2", StringType)
+
+    val rdd = spark.sparkContext.parallelize(0 to number, 2).map { item =>
+      val c1 = Integer.valueOf(item)
+      val c2 = s"abc"
+      RowFactory.create(c1, c2)
+    }
+    spark.createDataFrame(rdd, schema)
+  }
+
+  /**
+   * Intel(R) Xeon(R) Platinum 8259CL CPU @ 2.50GHz
+   * COW Ingestion:                            Best Time(ms)   Avg Time(ms)   
Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
+   * 
------------------------------------------------------------------------------------------------------------------------
+   * BoundInMemory Executor                             5557           5607    
      70          0.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)

Review Comment:
   👏 



##########
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

Review Comment:
   may I know what do we in this preExecute ? (as per this patch) 



##########
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();

Review Comment:
   In BoundedInMemoryExecutor, we start the producers and then start the 
consumer. do we know why was it done that way. Intuitive way is to start 
consumer and then stat producers. 



##########
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:
   guess thats how events are expected to be defined in disruptor framework. 
for eg 
https://lmax-exchange.github.io/disruptor/user-guide/index.html#_introduction 
check for LogEvent implementation here. 



##########
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,

Review Comment:
   may I know where is this cnstr used? 



##########
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:
   +1



##########
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:
   +1 



##########
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:
   +1



##########
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:
   +1 
   I see that we HoodieProducer is just an empty interface. and we have one 
abstract class for BoundedInMemory and another one for DistruptorBasedProducer. 
we should generify it and add produce() to base interface. 
   since HoodieMessageQueue interface is also shared among all producers and 
executor, we should definitely try to unify these. 
   



##########
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();

Review Comment:
   sorry I don't understand this. In BoundedInMemoryExecutor, we have a count 
down latch and wait for all producers to countdown and then close the queue. 
can you help explain what are we doing here. I guess we want to wait for all 
producers to complete before we can call close/shutdown. 



##########
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
+      .key("hoodie.write.wait.strategy")
+      .defaultValue("BlockingWaitStrategy")

Review Comment:
   not too strong on the suffix removal. feel free to take a call.



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

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

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

Reply via email to