1
0

Parallelized read-write operations in Hoodie Merge phase

This commit is contained in:
Nishith Agarwal
2018-04-01 21:43:05 -07:00
committed by vinoth chandar
parent 6c226ca21a
commit 720e42f52a
13 changed files with 553 additions and 126 deletions

View File

@@ -23,6 +23,8 @@ import com.uber.hoodie.common.HoodieTestDataGenerator;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
import com.uber.hoodie.exception.HoodieException;
import com.uber.hoodie.func.payload.AbstractBufferedIteratorPayload;
import com.uber.hoodie.func.payload.HoodieRecordBufferedIteratorPayload;
import java.io.IOException;
import java.util.Iterator;
import java.util.List;
@@ -66,7 +68,7 @@ public class TestBufferedIterator {
final int numRecords = 128;
final List<HoodieRecord> hoodieRecords = hoodieTestDataGenerator.generateInserts(commitTime, numRecords);
final BufferedIterator bufferedIterator = new BufferedIterator(hoodieRecords.iterator(), FileUtils.ONE_KB,
HoodieTestDataGenerator.avroSchema);
LazyInsertIterable.bufferedItrPayloadTransform(HoodieTestDataGenerator.avroSchema));
Future<Boolean> result = recordReader.submit(() -> {
bufferedIterator.startBuffering();
return true;
@@ -77,11 +79,12 @@ public class TestBufferedIterator {
final HoodieRecord originalRecord = originalRecordIterator.next();
final Optional<IndexedRecord> originalInsertValue = originalRecord.getData()
.getInsertValue(HoodieTestDataGenerator.avroSchema);
final BufferedIterator.BufferedIteratorPayload payload = bufferedIterator.next();
final HoodieRecordBufferedIteratorPayload payload = (HoodieRecordBufferedIteratorPayload) bufferedIterator.next();
// Ensure that record ordering is guaranteed.
Assert.assertEquals(originalRecord, payload.record);
Assert.assertEquals(originalRecord, payload.getInputPayload());
// cached insert value matches the expected insert value.
Assert.assertEquals(originalInsertValue, payload.insertValue);
Assert.assertEquals(originalInsertValue,
((HoodieRecord) payload.getInputPayload()).getData().getInsertValue(HoodieTestDataGenerator.avroSchema));
recordsRead++;
}
Assert.assertFalse(bufferedIterator.hasNext() || originalRecordIterator.hasNext());
@@ -99,8 +102,9 @@ public class TestBufferedIterator {
// maximum number of records to keep in memory.
final int recordLimit = 5;
final long memoryLimitInBytes = recordLimit * SizeEstimator.estimate(hoodieRecords.get(0));
final BufferedIterator bufferedIterator = new BufferedIterator(hoodieRecords.iterator(), memoryLimitInBytes,
HoodieTestDataGenerator.avroSchema);
final BufferedIterator<HoodieRecord, AbstractBufferedIteratorPayload> bufferedIterator =
new BufferedIterator(hoodieRecords.iterator(), memoryLimitInBytes,
LazyInsertIterable.bufferedItrPayloadTransform(HoodieTestDataGenerator.avroSchema));
Future<Boolean> result = recordReader.submit(() -> {
bufferedIterator.startBuffering();
return true;
@@ -115,8 +119,8 @@ public class TestBufferedIterator {
Assert.assertEquals(recordLimit - 1, bufferedIterator.samplingRecordCounter.get());
// try to read 2 records.
Assert.assertEquals(hoodieRecords.get(0), bufferedIterator.next().record);
Assert.assertEquals(hoodieRecords.get(1), bufferedIterator.next().record);
Assert.assertEquals(hoodieRecords.get(0), bufferedIterator.next().getInputPayload());
Assert.assertEquals(hoodieRecords.get(1), bufferedIterator.next().getInputPayload());
// waiting for permits to expire.
while (!isQueueFull(bufferedIterator.rateLimiter)) {
@@ -145,7 +149,7 @@ public class TestBufferedIterator {
// stops and throws
// correct exception back.
BufferedIterator bufferedIterator1 = new BufferedIterator(hoodieRecords.iterator(), memoryLimitInBytes,
HoodieTestDataGenerator.avroSchema);
LazyInsertIterable.bufferedItrPayloadTransform(HoodieTestDataGenerator.avroSchema));
Future<Boolean> result = recordReader.submit(() -> {
bufferedIterator1.startBuffering();
return true;
@@ -173,7 +177,7 @@ public class TestBufferedIterator {
when(mockHoodieRecordsIterator.hasNext()).thenReturn(true);
when(mockHoodieRecordsIterator.next()).thenThrow(expectedException);
BufferedIterator bufferedIterator2 = new BufferedIterator(mockHoodieRecordsIterator, memoryLimitInBytes,
HoodieTestDataGenerator.avroSchema);
LazyInsertIterable.bufferedItrPayloadTransform(HoodieTestDataGenerator.avroSchema));
Future<Boolean> result2 = recordReader.submit(() -> {
bufferedIterator2.startBuffering();
return true;

View File

@@ -0,0 +1,79 @@
/*
* Copyright (c) 2018 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
*
* Licensed 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 com.uber.hoodie.func;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
import com.uber.hoodie.common.HoodieTestDataGenerator;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
import com.uber.hoodie.config.HoodieWriteConfig;
import java.util.List;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.Future;
import java.util.function.Function;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
public class TestBufferedIteratorExecutor {
private final HoodieTestDataGenerator hoodieTestDataGenerator = new HoodieTestDataGenerator();
private final String commitTime = HoodieActiveTimeline.createNewCommitTime();
private ExecutorService executorService = null;
@Before
public void beforeTest() {
this.executorService = Executors.newFixedThreadPool(1);
}
@After
public void afterTest() {
if (this.executorService != null) {
this.executorService.shutdownNow();
this.executorService = null;
}
}
@Test
public void testExecutor() throws Exception {
final List<HoodieRecord> hoodieRecords = hoodieTestDataGenerator.generateInserts(commitTime, 100);
HoodieWriteConfig hoodieWriteConfig = mock(HoodieWriteConfig.class);
when(hoodieWriteConfig.getWriteBufferLimitBytes()).thenReturn(1024);
BufferedIteratorExecutor bufferedIteratorExecutor = new BufferedIteratorExecutor(hoodieWriteConfig,
hoodieRecords.iterator(), LazyInsertIterable.bufferedItrPayloadTransform(HoodieTestDataGenerator.avroSchema),
executorService);
Function<BufferedIterator, Integer> function = (bufferedIterator) -> {
Integer count = 0;
while (bufferedIterator.hasNext()) {
count++;
bufferedIterator.next();
}
return count;
};
Future<Integer> future = bufferedIteratorExecutor.start(function);
// It should buffer and write 100 records
Assert.assertEquals((int) future.get(), 100);
// There should be no remaining records in the buffer
Assert.assertFalse(bufferedIteratorExecutor.isRemaining());
}
}

View File

@@ -0,0 +1,61 @@
/*
* Copyright (c) 2018 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
*
* Licensed 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 com.uber.hoodie.func;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
import com.uber.hoodie.exception.HoodieIOException;
import java.io.IOException;
import org.apache.parquet.hadoop.ParquetReader;
import org.junit.Assert;
import org.junit.Test;
public class TestParquetReaderIterator {
@Test
public void testParquetIteratorIdempotency() throws IOException {
ParquetReader reader = mock(ParquetReader.class);
// only 1 record in reader
when(reader.read()).thenReturn(1).thenReturn(null);
ParquetReaderIterator<Integer> iterator = new ParquetReaderIterator<>(reader);
int idempotencyCheckCounter = 0;
// call hasNext() 3 times
while (idempotencyCheckCounter < 3) {
Assert.assertTrue(iterator.hasNext());
idempotencyCheckCounter++;
}
}
@Test
public void testParquetIterator() throws IOException {
ParquetReader reader = mock(ParquetReader.class);
// only one record to read
when(reader.read()).thenReturn(1).thenReturn(null);
ParquetReaderIterator<Integer> iterator = new ParquetReaderIterator<>(reader);
// should return value even though hasNext() hasn't been called
Assert.assertTrue(iterator.next() == 1);
// no more entries to iterate on
Assert.assertFalse(iterator.hasNext());
try {
iterator.next();
} catch (HoodieIOException e) {
// should throw an exception since there is only 1 record
}
}
}