-
Notifications
You must be signed in to change notification settings - Fork 2.5k
[HUDI-5023] Consuming records from Iterator directly instead of using inner message queue #7174
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Merged
alexeykudinkin
merged 22 commits into
apache:master
from
zhangyue19921010:consuming-records-directly-without-queue
Dec 15, 2022
Merged
Changes from all commits
Commits
Show all changes
22 commits
Select commit
Hold shift + click to select a range
5c1c1fd
ready to test
1e99dac
ready to test
37a6e2d
ready to test
cc37167
need more test
1e966be
add more java docs; still need to add more UTs
b4f4135
performance test
7a1e171
performance test
ec0c804
add more UTs
0cd32cd
finished benchmark
7ed45ff
fix checkstyle
178bf76
fix checkstyle
27316e8
fix checkstyle
e2e9203
refactor interface HoodieMessageQueue
a5c5fa8
rebase & change codes
bca3774
rebase
8f2f3a5
address comments
c734cef
address comments
db06b33
address comments
be53e07
address comments
783a5cd
address comments
74e0742
merge from master
b0c2730
address comments
File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
242 changes: 242 additions & 0 deletions
242
...hudi-spark-client/src/test/java/org/apache/hudi/execution/TestSimpleExecutionInSpark.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,242 @@ | ||
| /* | ||
| * 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.getCloningTransformer; | ||
|
|
||
| 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.HoodieConsumer; | ||
| import org.apache.hudi.common.util.queue.SimpleHoodieExecutor; | ||
| import org.apache.hudi.config.HoodieWriteConfig; | ||
| import org.apache.hudi.exception.HoodieException; | ||
| import org.apache.hudi.testutils.HoodieClientTestHarness; | ||
| import org.junit.jupiter.api.AfterEach; | ||
| import org.junit.jupiter.api.BeforeEach; | ||
| import org.junit.jupiter.api.Test; | ||
| import org.junit.jupiter.api.Timeout; | ||
|
|
||
| import java.io.IOException; | ||
| import java.util.ArrayList; | ||
| import java.util.Iterator; | ||
| import java.util.List; | ||
| import java.util.concurrent.atomic.AtomicInteger; | ||
|
|
||
| import scala.Tuple2; | ||
|
|
||
| import static org.junit.jupiter.api.Assertions.assertEquals; | ||
| import static org.junit.jupiter.api.Assertions.assertThrows; | ||
| import static org.junit.jupiter.api.Assertions.assertTrue; | ||
| import static org.mockito.Mockito.mock; | ||
| import static org.mockito.Mockito.when; | ||
|
|
||
| public class TestSimpleExecutionInSpark extends HoodieClientTestHarness { | ||
|
|
||
| private final String instantTime = HoodieActiveTimeline.createNewInstantTime(); | ||
|
|
||
| @BeforeEach | ||
| public void setUp() throws Exception { | ||
| initTestDataGenerator(); | ||
| initExecutorServiceWithFixedThreadPool(2); | ||
| } | ||
|
|
||
| @AfterEach | ||
| public void tearDown() throws Exception { | ||
| cleanupResources(); | ||
| } | ||
|
|
||
| @Test | ||
| public void testExecutor() { | ||
|
|
||
| final List<HoodieRecord> hoodieRecords = dataGen.generateInserts(instantTime, 128); | ||
| final List<HoodieRecord> consumedRecords = new ArrayList<>(); | ||
|
|
||
| HoodieWriteConfig hoodieWriteConfig = mock(HoodieWriteConfig.class); | ||
| when(hoodieWriteConfig.getDisruptorWriteBufferSize()).thenReturn(Option.of(8)); | ||
| HoodieConsumer<HoodieLazyInsertIterable.HoodieInsertValueGenResult<HoodieRecord>, Integer> consumer = | ||
| new HoodieConsumer<HoodieLazyInsertIterable.HoodieInsertValueGenResult<HoodieRecord>, Integer>() { | ||
| private int count = 0; | ||
|
|
||
| @Override | ||
| public void consume(HoodieLazyInsertIterable.HoodieInsertValueGenResult<HoodieRecord> record) throws Exception { | ||
| consumedRecords.add(record.getResult()); | ||
| count++; | ||
| } | ||
|
|
||
| @Override | ||
| public Integer finish() { | ||
| return count; | ||
| } | ||
| }; | ||
| SimpleHoodieExecutor<HoodieRecord, Tuple2<HoodieRecord, Option<IndexedRecord>>, Integer> exec = null; | ||
|
|
||
| try { | ||
| exec = new SimpleHoodieExecutor(hoodieRecords.iterator(), consumer, getCloningTransformer(HoodieTestDataGenerator.AVRO_SCHEMA)); | ||
|
|
||
| int result = exec.execute(); | ||
| // It should buffer and write 128 records | ||
| assertEquals(128, result); | ||
|
|
||
| // collect all records and assert that consumed records are identical to produced ones | ||
| // assert there's no tampering, and that the ordering is preserved | ||
| assertEquals(hoodieRecords, consumedRecords); | ||
|
|
||
| } finally { | ||
| if (exec != null) { | ||
| exec.shutdownNow(); | ||
| } | ||
| } | ||
| } | ||
|
|
||
| // 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() { | ||
|
|
||
| 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) { | ||
| // ignore exception here. | ||
| } | ||
| }); | ||
|
|
||
| HoodieConsumer<HoodieLazyInsertIterable.HoodieInsertValueGenResult<HoodieRecord>, Integer> consumer = | ||
| new HoodieConsumer<HoodieLazyInsertIterable.HoodieInsertValueGenResult<HoodieRecord>, Integer>() { | ||
| private int count = 0; | ||
|
|
||
| @Override | ||
| public void consume(HoodieLazyInsertIterable.HoodieInsertValueGenResult<HoodieRecord> record) throws Exception { | ||
| count++; | ||
| afterRecord.add((HoodieAvroRecord) record.getResult()); | ||
| try { | ||
| IndexedRecord indexedRecord = (IndexedRecord)((HoodieAvroRecord) record.getResult()) | ||
| .getData().getInsertValue(HoodieTestDataGenerator.AVRO_SCHEMA).get(); | ||
| afterIndexedRecord.add(indexedRecord); | ||
| } catch (IOException e) { | ||
| //ignore exception here. | ||
| } | ||
| } | ||
|
|
||
| @Override | ||
| public Integer finish() { | ||
| return count; | ||
| } | ||
| }; | ||
|
|
||
| SimpleHoodieExecutor<HoodieRecord, Tuple2<HoodieRecord, Option<IndexedRecord>>, Integer> exec = null; | ||
|
|
||
| try { | ||
| exec = new SimpleHoodieExecutor(hoodieRecords.iterator(), consumer, getCloningTransformer(HoodieTestDataGenerator.AVRO_SCHEMA)); | ||
| int result = exec.execute(); | ||
| assertEquals(100, result); | ||
|
|
||
| assertEquals(beforeRecord, afterRecord); | ||
| assertEquals(beforeIndexedRecord, afterIndexedRecord); | ||
|
|
||
| } finally { | ||
| if (exec != null) { | ||
| exec.shutdownNow(); | ||
| } | ||
| } | ||
| } | ||
|
|
||
| /** | ||
| * Test to ensure exception happend in iterator then we need to stop the simple ingestion. | ||
| */ | ||
| @SuppressWarnings("unchecked") | ||
| @Test | ||
| @Timeout(value = 60) | ||
| public void testException() { | ||
| final int numRecords = 1000; | ||
| final String errorMessage = "Exception when iterating records!!!"; | ||
|
|
||
| List<HoodieRecord> pRecs = dataGen.generateInserts(instantTime, numRecords); | ||
| InnerIterator iterator = new InnerIterator(pRecs.iterator(), errorMessage, numRecords / 10); | ||
|
|
||
| HoodieConsumer<HoodieLazyInsertIterable.HoodieInsertValueGenResult<HoodieRecord>, Integer> consumer = | ||
| new HoodieConsumer<HoodieLazyInsertIterable.HoodieInsertValueGenResult<HoodieRecord>, Integer>() { | ||
| int count = 0; | ||
|
|
||
| @Override | ||
| public void consume(HoodieLazyInsertIterable.HoodieInsertValueGenResult<HoodieRecord> payload) throws Exception { | ||
| // Read recs and ensure we have covered all producer recs. | ||
| final HoodieRecord rec = payload.getResult(); | ||
| count++; | ||
| } | ||
|
|
||
| @Override | ||
| public Integer finish() { | ||
| return count; | ||
| } | ||
| }; | ||
|
|
||
| SimpleHoodieExecutor<HoodieRecord, Tuple2<HoodieRecord, Option<IndexedRecord>>, Integer> exec = | ||
| new SimpleHoodieExecutor(iterator, consumer, getCloningTransformer(HoodieTestDataGenerator.AVRO_SCHEMA)); | ||
|
|
||
| final Throwable thrown = assertThrows(HoodieException.class, exec::execute, | ||
| "exception is expected"); | ||
| assertTrue(thrown.getMessage().contains(errorMessage)); | ||
| } | ||
|
|
||
| class InnerIterator implements Iterator<HoodieRecord> { | ||
|
|
||
| private Iterator<HoodieRecord> iterator; | ||
| private AtomicInteger count = new AtomicInteger(0); | ||
| private String errorMessage; | ||
| private int errorMessageCount; | ||
|
|
||
| public InnerIterator(Iterator<HoodieRecord> iterator, String errorMessage, int errorMessageCount) { | ||
| this.iterator = iterator; | ||
| this.errorMessage = errorMessage; | ||
| this.errorMessageCount = errorMessageCount; | ||
| } | ||
|
|
||
| @Override | ||
| public boolean hasNext() { | ||
| return iterator.hasNext(); | ||
| } | ||
|
|
||
| @Override | ||
| public HoodieRecord next() { | ||
| if (count.get() == errorMessageCount) { | ||
| throw new HoodieException(errorMessage); | ||
| } | ||
|
|
||
| count.incrementAndGet(); | ||
| return iterator.next(); | ||
| } | ||
| } | ||
| } | ||
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
Uh oh!
There was an error while loading. Please reload this page.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Hi @@alexeykudinkin
Not involved testInterruptExecutor in this SimpleExecutionSpark Test.
Bcz there is no inner Thread pool in this new SimpleExecutor so there is nothing to interrupt right?
Is that looks good to u ?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yeah, that makes sense