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


##########
hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorMessageQueue.java:
##########
@@ -0,0 +1,357 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.execution;
+
+import static 
org.apache.hudi.execution.HoodieLazyInsertIterable.getTransformFunction;
+
+import org.apache.avro.generic.IndexedRecord;
+import org.apache.hudi.common.model.HoodieAvroRecord;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
+import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.queue.DisruptorMessageHandler;
+import org.apache.hudi.common.util.queue.DisruptorMessageQueue;
+import org.apache.hudi.common.util.queue.DisruptorPublisher;
+import org.apache.hudi.common.util.queue.FunctionBasedQueueProducer;
+import org.apache.hudi.common.util.queue.HoodieProducer;
+import org.apache.hudi.common.util.queue.IteratorBasedQueueConsumer;
+import org.apache.hudi.common.util.queue.DisruptorExecutor;
+import org.apache.hudi.common.util.queue.IteratorBasedQueueProducer;
+import org.apache.hudi.common.util.queue.WaitStrategyFactory;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.testutils.HoodieClientTestHarness;
+import org.apache.spark.TaskContext;
+import org.apache.spark.TaskContext$;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.Future;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+import scala.Tuple2;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+public class TestDisruptorMessageQueue extends HoodieClientTestHarness {
+
+  private final String instantTime = 
HoodieActiveTimeline.createNewInstantTime();
+
+  @BeforeEach
+  public void setUp() throws Exception {
+    initTestDataGenerator();
+    initExecutorServiceWithFixedThreadPool(2);
+  }
+
+  @AfterEach
+  public void tearDown() throws Exception {
+    cleanupResources();
+  }
+
+  private Runnable getPreExecuteRunnable() {
+    final TaskContext taskContext = TaskContext.get();
+    return () -> TaskContext$.MODULE$.setTaskContext(taskContext);
+  }
+
+  // Test to ensure that we are reading all records from queue iterator in the 
same order
+  // without any exceptions.
+  @SuppressWarnings("unchecked")
+  @Test
+  @Timeout(value = 60)
+  public void testRecordReading() throws Exception {
+
+    final List<HoodieRecord> hoodieRecords = 
dataGen.generateInserts(instantTime, 100);
+    ArrayList<HoodieRecord> beforeRecord = new ArrayList<>();
+    ArrayList<IndexedRecord> beforeIndexedRecord = new ArrayList<>();
+    ArrayList<HoodieAvroRecord> afterRecord = new ArrayList<>();
+    ArrayList<IndexedRecord> afterIndexedRecord = new ArrayList<>();
+
+    hoodieRecords.forEach(record -> {
+      final HoodieAvroRecord originalRecord = (HoodieAvroRecord) record;
+      beforeRecord.add(originalRecord);
+      try {
+        final Option<IndexedRecord> originalInsertValue =
+            
originalRecord.getData().getInsertValue(HoodieTestDataGenerator.AVRO_SCHEMA);
+        beforeIndexedRecord.add(originalInsertValue.get());
+      } catch (IOException e) {
+        e.printStackTrace();
+      }
+    });
+
+    HoodieWriteConfig hoodieWriteConfig = mock(HoodieWriteConfig.class);
+    when(hoodieWriteConfig.getWriteBufferSize()).thenReturn(16);
+    
IteratorBasedQueueConsumer<HoodieLazyInsertIterable.HoodieInsertValueGenResult<HoodieRecord>,
 Integer> consumer =
+        new 
IteratorBasedQueueConsumer<HoodieLazyInsertIterable.HoodieInsertValueGenResult<HoodieRecord>,
 Integer>() {
+
+          private int count = 0;
+
+          @Override
+          public void 
consumeOneRecord(HoodieLazyInsertIterable.HoodieInsertValueGenResult<HoodieRecord>
 record) {
+            count++;
+            afterRecord.add((HoodieAvroRecord) record.record);
+            try {
+              IndexedRecord indexedRecord = (IndexedRecord)((HoodieAvroRecord) 
record.record)
+                  
.getData().getInsertValue(HoodieTestDataGenerator.AVRO_SCHEMA).get();
+              afterIndexedRecord.add(indexedRecord);
+            } catch (IOException e) {
+              e.printStackTrace();
+            }
+          }
+
+          @Override
+          public void finish() {
+          }
+
+          @Override
+          protected Integer getResult() {
+            return count;
+          }
+    };
+
+    DisruptorExecutor<HoodieRecord, Tuple2<HoodieRecord, 
Option<IndexedRecord>>, Integer> exec = null;
+
+    try {
+      exec = new DisruptorExecutor(hoodieWriteConfig.getWriteBufferSize(), 
hoodieRecords.iterator(), consumer,
+          getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA), 
WaitStrategyFactory.DEFAULT_STRATEGY, getPreExecuteRunnable());
+      int result = exec.execute();
+      // It should buffer and write 100 records
+      assertEquals(100, result);
+      // There should be no remaining records in the buffer
+      assertFalse(exec.isRemaining());
+
+      assertEquals(beforeRecord, afterRecord);
+      assertEquals(beforeIndexedRecord, afterIndexedRecord);
+
+    } finally {
+      if (exec != null) {
+        exec.shutdownNow();
+      }
+    }
+  }
+
+  /**
+   * Test to ensure that we are reading all records from queue iterator when 
we have multiple producers.
+   */
+  @SuppressWarnings("unchecked")
+  @Test
+  @Timeout(value = 60)
+  public void testCompositeProducerRecordReading() throws Exception {
+    final int numRecords = 1000;
+    final int numProducers = 40;
+    final List<List<HoodieRecord>> recs = new ArrayList<>();
+
+    final DisruptorMessageQueue<HoodieRecord, 
HoodieLazyInsertIterable.HoodieInsertValueGenResult> queue =
+        new DisruptorMessageQueue(1024, 
getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA),
+            "BLOCKING_WAIT", numProducers, new Runnable() {
+              @Override
+          public void run() {
+                // do nothing.
+              }
+            });
+
+    // Record Key to <Producer Index, Rec Index within a producer>
+    Map<String, Tuple2<Integer, Integer>> keyToProducerAndIndexMap = new 
HashMap<>();
+
+    for (int i = 0; i < numProducers; i++) {
+      List<HoodieRecord> pRecs = dataGen.generateInserts(instantTime, 
numRecords);
+      int j = 0;
+      for (HoodieRecord r : pRecs) {
+        assertFalse(keyToProducerAndIndexMap.containsKey(r.getRecordKey()));
+        keyToProducerAndIndexMap.put(r.getRecordKey(), new Tuple2<>(i, j));
+        j++;
+      }
+      recs.add(pRecs);
+    }
+
+    List<DisruptorPublisher> disruptorPublishers = new ArrayList<>();
+    for (int i = 0; i < recs.size(); i++) {
+      final List<HoodieRecord> r = recs.get(i);
+      // Alternate between pull and push based iterators
+      if (i % 2 == 0) {
+        DisruptorPublisher publisher = new DisruptorPublisher<>(new 
IteratorBasedQueueProducer<>(r.iterator()), queue);
+        disruptorPublishers.add(publisher);
+      } else {
+        DisruptorPublisher publisher = new DisruptorPublisher<>(new 
FunctionBasedQueueProducer<>((buf) -> {
+          Iterator<HoodieRecord> itr = r.iterator();
+          while (itr.hasNext()) {
+            try {
+              buf.insertRecord(itr.next());
+            } catch (Exception e) {
+              throw new HoodieException(e);
+            }
+          }
+          return true;
+        }), queue);
+        disruptorPublishers.add(publisher);
+      }
+    }
+
+    // Used to ensure that consumer sees the records generated by a single 
producer in FIFO order
+    Map<Integer, Integer> lastSeenMap =
+        IntStream.range(0, 
numProducers).boxed().collect(Collectors.toMap(Function.identity(), x -> -1));
+    Map<Integer, Integer> countMap =
+        IntStream.range(0, 
numProducers).boxed().collect(Collectors.toMap(Function.identity(), x -> 0));
+
+
+    // setup consumer and start disruptor
+    
DisruptorMessageHandler<HoodieLazyInsertIterable.HoodieInsertValueGenResult<HoodieRecord>,
 Integer> handler =
+        new DisruptorMessageHandler<>(new 
IteratorBasedQueueConsumer<HoodieLazyInsertIterable.HoodieInsertValueGenResult<HoodieRecord>,
 Integer>() {
+
+          @Override
+          public void 
consumeOneRecord(HoodieLazyInsertIterable.HoodieInsertValueGenResult<HoodieRecord>
 payload) {
+            // Read recs and ensure we have covered all producer recs.
+            final HoodieRecord rec = payload.record;
+            Tuple2<Integer, Integer> producerPos = 
keyToProducerAndIndexMap.get(rec.getRecordKey());
+            Integer lastSeenPos = lastSeenMap.get(producerPos._1());
+            countMap.put(producerPos._1(), countMap.get(producerPos._1()) + 1);
+            lastSeenMap.put(producerPos._1(), lastSeenPos + 1);
+            // Ensure we are seeing the next record generated
+            assertEquals(lastSeenPos + 1, producerPos._2().intValue());

Review Comment:
   We can only guarantee that the data of a single producer is in order, but 
the data sent by multiple producers to the queue is out of order. So that here 
use a map to collect and check the value and order for each producer.
   Also this is follow the original design of 
`org.apache.hudi.execution#testCompositeProducerRecordReading`



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

To unsubscribe, e-mail: [email protected]

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

Reply via email to