1
0

[HUDI-431] Adding support for Parquet in MOR LogBlocks (#4333)

- Adding support for Parquet in MOR tables Log blocks

Co-authored-by: Sivabalan Narayanan <n.siva.b@gmail.com>
This commit is contained in:
Alexey Kudinkin
2022-02-02 11:35:05 -08:00
committed by GitHub
parent caef3d5c58
commit a68e1dc2db
45 changed files with 1613 additions and 670 deletions

View File

@@ -21,10 +21,12 @@ package org.apache.hudi.common.functional;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.io.compress.Compression;
import org.apache.hudi.avro.HoodieAvroUtils;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.HoodieArchivedLogFile;
@@ -46,7 +48,9 @@ import org.apache.hudi.common.table.log.block.HoodieHFileDataBlock;
import org.apache.hudi.common.table.log.block.HoodieLogBlock;
import org.apache.hudi.common.table.log.block.HoodieLogBlock.HeaderMetadataType;
import org.apache.hudi.common.table.log.block.HoodieLogBlock.HoodieLogBlockType;
import org.apache.hudi.common.table.log.block.HoodieParquetDataBlock;
import org.apache.hudi.common.testutils.FileCreateUtils;
import org.apache.hudi.common.testutils.HadoopMapRedUtils;
import org.apache.hudi.common.testutils.HoodieCommonTestHarness;
import org.apache.hudi.common.testutils.HoodieTestUtils;
import org.apache.hudi.common.testutils.SchemaTestUtil;
@@ -54,8 +58,9 @@ import org.apache.hudi.common.testutils.minicluster.MiniClusterUtil;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.collection.ExternalSpillableMap;
import org.apache.hudi.exception.CorruptedLogFileException;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.parquet.hadoop.metadata.CompressionCodecName;
import org.apache.parquet.hadoop.util.counters.BenchmarkCounter;
import org.junit.jupiter.api.AfterAll;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeAll;
@@ -96,11 +101,12 @@ import static org.junit.jupiter.params.provider.Arguments.arguments;
@SuppressWarnings("Duplicates")
public class TestHoodieLogFormat extends HoodieCommonTestHarness {
private static final HoodieLogBlockType DEFAULT_DATA_BLOCK_TYPE = HoodieLogBlockType.AVRO_DATA_BLOCK;
private static String BASE_OUTPUT_PATH = "/tmp/";
private FileSystem fs;
private Path partitionPath;
private int bufferSize = 4096;
private HoodieLogBlockType dataBlockType = HoodieLogBlockType.AVRO_DATA_BLOCK;
@BeforeAll
public static void setUpClass() throws IOException, InterruptedException {
@@ -139,7 +145,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
}
@ParameterizedTest
@EnumSource(names = { "AVRO_DATA_BLOCK", "HFILE_DATA_BLOCK" })
@EnumSource(names = {"AVRO_DATA_BLOCK", "HFILE_DATA_BLOCK", "PARQUET_DATA_BLOCK"})
public void testBasicAppend(HoodieLogBlockType dataBlockType) throws IOException, InterruptedException, URISyntaxException {
Writer writer =
HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath).withFileExtension(HoodieLogFile.DELTA_EXTENSION)
@@ -171,7 +177,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
Map<HoodieLogBlock.HeaderMetadataType, String> header = new HashMap<>();
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString());
HoodieDataBlock dataBlock = getDataBlock(records, header);
HoodieDataBlock dataBlock = getDataBlock(DEFAULT_DATA_BLOCK_TYPE, records, header);
// Write out a block
AppendResult firstAppend = writer.appendBlock(dataBlock);
// Get the size of the block
@@ -186,7 +192,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath).withFileExtension(HoodieLogFile.DELTA_EXTENSION)
.withFileId("test-fileid1").overBaseCommit("100").withFs(fs).withSizeThreshold(size - 1).build();
records = SchemaTestUtil.generateTestRecords(0, 100);
dataBlock = getDataBlock(records, header);
dataBlock = getDataBlock(DEFAULT_DATA_BLOCK_TYPE, records, header);
AppendResult secondAppend = writer.appendBlock(dataBlock);
assertEquals(firstAppend.logFile(), secondAppend.logFile());
@@ -198,7 +204,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
// Write one more block, which should not go to the new log file.
records = SchemaTestUtil.generateTestRecords(0, 100);
dataBlock = getDataBlock(records, header);
dataBlock = getDataBlock(DEFAULT_DATA_BLOCK_TYPE, records, header);
AppendResult rolloverAppend = writer.appendBlock(dataBlock);
assertNotEquals(secondAppend.logFile(), rolloverAppend.logFile());
@@ -245,7 +251,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
Map<HoodieLogBlock.HeaderMetadataType, String> header = new HashMap<>();
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString());
HoodieDataBlock dataBlock = getDataBlock(records, header);
HoodieDataBlock dataBlock = getDataBlock(DEFAULT_DATA_BLOCK_TYPE, records, header);
writer.appendBlock(dataBlock);
Writer writer2 = builder2.build();
writer2.appendBlock(dataBlock);
@@ -257,8 +263,9 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
assertEquals(logFile1.getLogVersion(), logFile2.getLogVersion() - 1, "Log Files must have different versions");
}
@Test
public void testMultipleAppend() throws IOException, URISyntaxException, InterruptedException {
@ParameterizedTest
@EnumSource(names = {"AVRO_DATA_BLOCK", "HFILE_DATA_BLOCK", "PARQUET_DATA_BLOCK"})
public void testMultipleAppend(HoodieLogBlockType dataBlockType) throws IOException, URISyntaxException, InterruptedException {
Writer writer =
HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath).withFileExtension(HoodieLogFile.DELTA_EXTENSION)
.withFileId("test-fileid1").overBaseCommit("100").withFs(fs).build();
@@ -266,7 +273,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
Map<HoodieLogBlock.HeaderMetadataType, String> header = new HashMap<>();
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString());
HoodieDataBlock dataBlock = getDataBlock(records, header);
HoodieDataBlock dataBlock = getDataBlock(dataBlockType, records, header);
writer.appendBlock(dataBlock);
long size1 = writer.getCurrentSize();
writer.close();
@@ -276,7 +283,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
.withFileId("test-fileid1").overBaseCommit("100").withFs(fs).build();
records = SchemaTestUtil.generateTestRecords(0, 100);
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString());
dataBlock = getDataBlock(records, header);
dataBlock = getDataBlock(dataBlockType, records, header);
writer.appendBlock(dataBlock);
long size2 = writer.getCurrentSize();
assertTrue(size2 > size1, "We just wrote a new block - size2 should be > size1");
@@ -290,7 +297,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
.withFileId("test-fileid1").overBaseCommit("100").withFs(fs).build();
records = SchemaTestUtil.generateTestRecords(0, 100);
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString());
dataBlock = getDataBlock(records, header);
dataBlock = getDataBlock(dataBlockType, records, header);
writer.appendBlock(dataBlock);
long size3 = writer.getCurrentSize();
assertTrue(size3 > size2, "We just wrote a new block - size3 should be > size2");
@@ -309,26 +316,27 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
* This is actually a test on concurrent append and not recovery lease. Commenting this out.
* https://issues.apache.org/jira/browse/HUDI-117
*/
/**
* @Test public void testLeaseRecovery() throws IOException, URISyntaxException, InterruptedException { Writer writer
* = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath)
* .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
* .overBaseCommit("100").withFs(fs).build(); List<IndexedRecord> records =
* SchemaTestUtil.generateTestRecords(0, 100); Map<HoodieLogBlock.HeaderMetadataType, String> header =
* Maps.newHashMap(); header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
* header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString()); HoodieAvroDataBlock
* dataBlock = new HoodieAvroDataBlock(records, header); writer = writer.appendBlock(dataBlock); long size1 =
* writer.getCurrentSize(); // do not close this writer - this simulates a data note appending to a log dying
* without closing the file // writer.close();
*
* writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath)
* .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1").overBaseCommit("100")
* .withFs(fs).build(); records = SchemaTestUtil.generateTestRecords(0, 100);
* header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString()); dataBlock = new
* HoodieAvroDataBlock(records, header); writer = writer.appendBlock(dataBlock); long size2 =
* writer.getCurrentSize(); assertTrue("We just wrote a new block - size2 should be > size1", size2 > size1);
* assertEquals("Write should be auto-flushed. The size reported by FileStatus and the writer should match",
* size2, fs.getFileStatus(writer.getLogFile().getPath()).getLen()); writer.close(); }
* = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath)
* .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
* .overBaseCommit("100").withFs(fs).build(); List<IndexedRecord> records =
* SchemaTestUtil.generateTestRecords(0, 100); Map<HoodieLogBlock.HeaderMetadataType, String> header =
* Maps.newHashMap(); header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
* header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString()); HoodieAvroDataBlock
* dataBlock = new HoodieAvroDataBlock(records, header); writer = writer.appendBlock(dataBlock); long size1 =
* writer.getCurrentSize(); // do not close this writer - this simulates a data note appending to a log dying
* without closing the file // writer.close();
* <p>
* writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath)
* .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1").overBaseCommit("100")
* .withFs(fs).build(); records = SchemaTestUtil.generateTestRecords(0, 100);
* header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString()); dataBlock = new
* HoodieAvroDataBlock(records, header); writer = writer.appendBlock(dataBlock); long size2 =
* writer.getCurrentSize(); assertTrue("We just wrote a new block - size2 should be > size1", size2 > size1);
* assertEquals("Write should be auto-flushed. The size reported by FileStatus and the writer should match",
* size2, fs.getFileStatus(writer.getLogFile().getPath()).getLen()); writer.close(); }
*/
@Test
@@ -344,7 +352,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
Map<HoodieLogBlock.HeaderMetadataType, String> header = new HashMap<>();
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString());
HoodieDataBlock dataBlock = getDataBlock(records, header);
HoodieDataBlock dataBlock = getDataBlock(DEFAULT_DATA_BLOCK_TYPE, records, header);
for (int i = 0; i < 2; i++) {
Writer writer = HoodieLogFormat.newWriterBuilder().onParentPath(testPath)
@@ -371,14 +379,14 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
Map<HoodieLogBlock.HeaderMetadataType, String> header = new HashMap<>();
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString());
HoodieDataBlock dataBlock = getDataBlock(records, header);
HoodieDataBlock dataBlock = getDataBlock(DEFAULT_DATA_BLOCK_TYPE, records, header);
writer.appendBlock(dataBlock);
writer.close();
Reader reader = HoodieLogFormat.newReader(fs, writer.getLogFile(), SchemaTestUtil.getSimpleSchema());
assertTrue(reader.hasNext(), "We wrote a block, we should be able to read it");
HoodieLogBlock nextBlock = reader.next();
assertEquals(dataBlockType, nextBlock.getBlockType(), "The next block should be a data block");
assertEquals(DEFAULT_DATA_BLOCK_TYPE, nextBlock.getBlockType(), "The next block should be a data block");
HoodieDataBlock dataBlockRead = (HoodieDataBlock) nextBlock;
assertEquals(copyOfRecords.size(), dataBlockRead.getRecords().size(),
"Read records size should be equal to the written records size");
@@ -400,10 +408,10 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
Map<HoodieLogBlock.HeaderMetadataType, String> header = new HashMap<>();
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString());
byte[] dataBlockContentBytes = getDataBlock(records, header).getContentBytes();
HoodieDataBlock reusableDataBlock = new HoodieAvroDataBlock(null, null,
Option.ofNullable(dataBlockContentBytes), false, 0, dataBlockContentBytes.length,
0, getSimpleSchema(), header, new HashMap<>(), HoodieRecord.RECORD_KEY_METADATA_FIELD);
byte[] dataBlockContentBytes = getDataBlock(DEFAULT_DATA_BLOCK_TYPE, records, header).getContentBytes();
HoodieLogBlock.HoodieLogBlockContentLocation logBlockContentLoc = new HoodieLogBlock.HoodieLogBlockContentLocation(new Configuration(), null, 0, dataBlockContentBytes.length, 0);
HoodieDataBlock reusableDataBlock = new HoodieAvroDataBlock(null, Option.ofNullable(dataBlockContentBytes), false,
logBlockContentLoc, Option.ofNullable(getSimpleSchema()), header, new HashMap<>(), HoodieRecord.RECORD_KEY_METADATA_FIELD);
long writtenSize = 0;
int logBlockWrittenNum = 0;
while (writtenSize < Integer.MAX_VALUE) {
@@ -418,7 +426,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
true, true);
assertTrue(reader.hasNext(), "We wrote a block, we should be able to read it");
HoodieLogBlock nextBlock = reader.next();
assertEquals(dataBlockType, nextBlock.getBlockType(), "The next block should be a data block");
assertEquals(DEFAULT_DATA_BLOCK_TYPE, nextBlock.getBlockType(), "The next block should be a data block");
HoodieDataBlock dataBlockRead = (HoodieDataBlock) nextBlock;
assertEquals(copyOfRecords.size(), dataBlockRead.getRecords().size(),
"Read records size should be equal to the written records size");
@@ -447,11 +455,16 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
oversizeWriter.close();
}
@Test
public void testBasicAppendAndRead() throws IOException, URISyntaxException, InterruptedException {
Writer writer =
HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath).withFileExtension(HoodieLogFile.DELTA_EXTENSION)
.withFileId("test-fileid1").overBaseCommit("100").withFs(fs).build();
@ParameterizedTest
@EnumSource(names = {"AVRO_DATA_BLOCK", "HFILE_DATA_BLOCK", "PARQUET_DATA_BLOCK"})
public void testBasicAppendAndRead(HoodieLogBlockType dataBlockType) throws IOException, URISyntaxException, InterruptedException {
Writer writer = HoodieLogFormat.newWriterBuilder()
.onParentPath(partitionPath)
.withFileExtension(HoodieLogFile.DELTA_EXTENSION)
.withFileId("test-fileid1")
.overBaseCommit("100")
.withFs(fs)
.build();
List<IndexedRecord> records1 = SchemaTestUtil.generateTestRecords(0, 100);
Schema schema = getSimpleSchema();
List<IndexedRecord> copyOfRecords1 = records1.stream()
@@ -459,30 +472,39 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
Map<HoodieLogBlock.HeaderMetadataType, String> header = new HashMap<>();
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString());
HoodieDataBlock dataBlock = getDataBlock(records1, header);
HoodieDataBlock dataBlock = getDataBlock(dataBlockType, records1, header);
writer.appendBlock(dataBlock);
writer.close();
writer =
HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath).withFileExtension(HoodieLogFile.DELTA_EXTENSION)
.withFileId("test-fileid1").overBaseCommit("100").withFs(fs).build();
writer = HoodieLogFormat.newWriterBuilder()
.onParentPath(partitionPath)
.withFileExtension(HoodieLogFile.DELTA_EXTENSION)
.withFileId("test-fileid1")
.overBaseCommit("100")
.withFs(fs)
.build();
List<IndexedRecord> records2 = SchemaTestUtil.generateTestRecords(0, 100);
List<IndexedRecord> copyOfRecords2 = records2.stream()
.map(record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList());
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString());
dataBlock = getDataBlock(records2, header);
dataBlock = getDataBlock(dataBlockType, records2, header);
writer.appendBlock(dataBlock);
writer.close();
// Close and Open again and append 100 more records
writer =
HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath).withFileExtension(HoodieLogFile.DELTA_EXTENSION)
.withFileId("test-fileid1").overBaseCommit("100").withFs(fs).build();
writer = HoodieLogFormat.newWriterBuilder()
.onParentPath(partitionPath)
.withFileExtension(HoodieLogFile.DELTA_EXTENSION)
.withFileId("test-fileid1")
.overBaseCommit("100")
.withFs(fs)
.build();
List<IndexedRecord> records3 = SchemaTestUtil.generateTestRecords(0, 100);
List<IndexedRecord> copyOfRecords3 = records3.stream()
.map(record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList());
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString());
dataBlock = getDataBlock(records3, header);
dataBlock = getDataBlock(dataBlockType, records3, header);
writer.appendBlock(dataBlock);
writer.close();
@@ -538,7 +560,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
.map(record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList());
allRecords.add(copyOfRecords1);
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
HoodieDataBlock dataBlock = getDataBlock(records1, header);
HoodieDataBlock dataBlock = getDataBlock(DEFAULT_DATA_BLOCK_TYPE, records1, header);
writer.appendBlock(dataBlock);
}
writer.close();
@@ -580,7 +602,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
Map<HoodieLogBlock.HeaderMetadataType, String> header = new HashMap<>();
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString());
HoodieDataBlock dataBlock = getDataBlock(records, header);
HoodieDataBlock dataBlock = getDataBlock(DEFAULT_DATA_BLOCK_TYPE, records, header);
writer.appendBlock(dataBlock);
writer.close();
@@ -602,11 +624,11 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
// Append a proper block that is of the missing length of the corrupted block
writer =
HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath).withFileExtension(HoodieLogFile.DELTA_EXTENSION)
.withFileId("test-fileid1").overBaseCommit("100").withFs(fs).build();
HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath).withFileExtension(HoodieLogFile.DELTA_EXTENSION)
.withFileId("test-fileid1").overBaseCommit("100").withFs(fs).build();
records = SchemaTestUtil.generateTestRecords(0, 10);
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString());
dataBlock = getDataBlock(records, header);
dataBlock = getDataBlock(DEFAULT_DATA_BLOCK_TYPE, records, header);
writer.appendBlock(dataBlock);
writer.close();
@@ -644,7 +666,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
.withFileId("test-fileid1").overBaseCommit("100").withFs(fs).build();
records = SchemaTestUtil.generateTestRecords(0, 100);
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString());
dataBlock = getDataBlock(records, header);
dataBlock = getDataBlock(DEFAULT_DATA_BLOCK_TYPE, records, header);
writer.appendBlock(dataBlock);
writer.close();
@@ -674,7 +696,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
Map<HoodieLogBlock.HeaderMetadataType, String> header = new HashMap<>();
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString());
HoodieDataBlock dataBlock = getDataBlock(records, header);
HoodieDataBlock dataBlock = getDataBlock(DEFAULT_DATA_BLOCK_TYPE, records, header);
writer.appendBlock(dataBlock);
writer.close();
@@ -702,7 +724,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
.withFileId("test-fileid1").overBaseCommit("100").withFs(fs).build();
records = SchemaTestUtil.generateTestRecords(0, 10);
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString());
dataBlock = getDataBlock(records, header);
dataBlock = getDataBlock(DEFAULT_DATA_BLOCK_TYPE, records, header);
writer.appendBlock(dataBlock);
writer.close();
@@ -741,7 +763,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
Map<HoodieLogBlock.HeaderMetadataType, String> header = new HashMap<>();
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
HoodieDataBlock dataBlock = getDataBlock(records1, header);
HoodieDataBlock dataBlock = getDataBlock(DEFAULT_DATA_BLOCK_TYPE, records1, header);
writer.appendBlock(dataBlock);
// Write 2
@@ -749,7 +771,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
List<IndexedRecord> copyOfRecords2 = records2.stream()
.map(record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList());
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
dataBlock = getDataBlock(records2, header);
dataBlock = getDataBlock(DEFAULT_DATA_BLOCK_TYPE, records2, header);
writer.appendBlock(dataBlock);
writer.close();
@@ -804,14 +826,14 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
HoodieDataBlock dataBlock = getDataBlock(records1, header);
HoodieDataBlock dataBlock = getDataBlock(DEFAULT_DATA_BLOCK_TYPE, records1, header);
writer.appendBlock(dataBlock);
// Write 2
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "101");
List<IndexedRecord> records2 = SchemaTestUtil.generateHoodieTestRecords(0, 100);
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
dataBlock = getDataBlock(records2, header);
dataBlock = getDataBlock(DEFAULT_DATA_BLOCK_TYPE, records2, header);
writer.appendBlock(dataBlock);
// Rollback the last write
@@ -827,7 +849,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
List<IndexedRecord> copyOfRecords3 = records3.stream()
.map(record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList());
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
dataBlock = getDataBlock(records3, header);
dataBlock = getDataBlock(DEFAULT_DATA_BLOCK_TYPE, records3, header);
writer.appendBlock(dataBlock);
writer.close();
@@ -880,7 +902,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
Map<HoodieLogBlock.HeaderMetadataType, String> header = new HashMap<>();
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
HoodieDataBlock dataBlock = getDataBlock(records1, header);
HoodieDataBlock dataBlock = getDataBlock(DEFAULT_DATA_BLOCK_TYPE, records1, header);
writer.appendBlock(dataBlock);
writer.close();
@@ -914,7 +936,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
.map(record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList());
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
dataBlock = getDataBlock(records3, header);
dataBlock = getDataBlock(DEFAULT_DATA_BLOCK_TYPE, records3, header);
writer.appendBlock(dataBlock);
writer.close();
@@ -968,7 +990,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
Map<HoodieLogBlock.HeaderMetadataType, String> header = new HashMap<>();
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
HoodieDataBlock dataBlock = getDataBlock(records1, header);
HoodieDataBlock dataBlock = getDataBlock(DEFAULT_DATA_BLOCK_TYPE, records1, header);
writer.appendBlock(dataBlock);
// Write 2
@@ -976,7 +998,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
List<IndexedRecord> records2 = SchemaTestUtil.generateHoodieTestRecords(0, 100);
List<IndexedRecord> copyOfRecords2 = records2.stream()
.map(record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList());
dataBlock = getDataBlock(records2, header);
dataBlock = getDataBlock(DEFAULT_DATA_BLOCK_TYPE, records2, header);
writer.appendBlock(dataBlock);
copyOfRecords1.addAll(copyOfRecords2);
@@ -1089,13 +1111,13 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
header.put(HoodieLogBlock.HeaderMetadataType.TARGET_INSTANT_TIME, "100");
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
HoodieDataBlock dataBlock = getDataBlock(records1, header);
HoodieDataBlock dataBlock = getDataBlock(DEFAULT_DATA_BLOCK_TYPE, records1, header);
writer.appendBlock(dataBlock);
// Write 2
List<IndexedRecord> records2 = SchemaTestUtil.generateHoodieTestRecords(0, 100);
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
dataBlock = getDataBlock(records2, header);
dataBlock = getDataBlock(DEFAULT_DATA_BLOCK_TYPE, records2, header);
writer.appendBlock(dataBlock);
// Delete 50 keys
@@ -1173,7 +1195,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
header.put(HoodieLogBlock.HeaderMetadataType.TARGET_INSTANT_TIME, "100");
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
HoodieDataBlock dataBlock = getDataBlock(records1, header);
HoodieDataBlock dataBlock = getDataBlock(DEFAULT_DATA_BLOCK_TYPE, records1, header);
writer.appendBlock(dataBlock);
// Delete 50 keys
@@ -1232,7 +1254,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
Map<HoodieLogBlock.HeaderMetadataType, String> header = new HashMap<>();
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
HoodieDataBlock dataBlock = getDataBlock(records1, header);
HoodieDataBlock dataBlock = getDataBlock(DEFAULT_DATA_BLOCK_TYPE, records1, header);
writer.appendBlock(dataBlock);
FileCreateUtils.createDeltaCommit(basePath, "100", fs);
@@ -1290,7 +1312,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
header.put(HoodieLogBlock.HeaderMetadataType.TARGET_INSTANT_TIME, "100");
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
HoodieDataBlock dataBlock = getDataBlock(records1, header);
HoodieDataBlock dataBlock = getDataBlock(DEFAULT_DATA_BLOCK_TYPE, records1, header);
writer.appendBlock(dataBlock);
writer.appendBlock(dataBlock);
writer.appendBlock(dataBlock);
@@ -1354,7 +1376,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
Map<HoodieLogBlock.HeaderMetadataType, String> header = new HashMap<>();
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
HoodieDataBlock dataBlock = getDataBlock(records1, header);
HoodieDataBlock dataBlock = getDataBlock(DEFAULT_DATA_BLOCK_TYPE, records1, header);
writer.appendBlock(dataBlock);
writer.appendBlock(dataBlock);
writer.appendBlock(dataBlock);
@@ -1473,7 +1495,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
Map<HoodieLogBlock.HeaderMetadataType, String> header = new HashMap<>();
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
HoodieDataBlock dataBlock = getDataBlock(records.subList(0, numRecordsInLog1), header);
HoodieDataBlock dataBlock = getDataBlock(DEFAULT_DATA_BLOCK_TYPE, records.subList(0, numRecordsInLog1), header);
writer.appendBlock(dataBlock);
// Get the size of the block
long size = writer.getCurrentSize();
@@ -1487,7 +1509,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
Map<HoodieLogBlock.HeaderMetadataType, String> header2 = new HashMap<>();
header2.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
header2.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
HoodieDataBlock dataBlock2 = getDataBlock(records2.subList(0, numRecordsInLog2), header2);
HoodieDataBlock dataBlock2 = getDataBlock(DEFAULT_DATA_BLOCK_TYPE, records2.subList(0, numRecordsInLog2), header2);
writer2.appendBlock(dataBlock2);
// Get the size of the block
writer2.close();
@@ -1574,7 +1596,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
Map<HoodieLogBlock.HeaderMetadataType, String> header = new HashMap<>();
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
HoodieDataBlock dataBlock = getDataBlock(records1, header);
HoodieDataBlock dataBlock = getDataBlock(DEFAULT_DATA_BLOCK_TYPE, records1, header);
writer.appendBlock(dataBlock);
writer.close();
@@ -1584,7 +1606,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
List<IndexedRecord> records2 = SchemaTestUtil.generateTestRecords(0, 100);
List<IndexedRecord> copyOfRecords2 = records2.stream()
.map(record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList());
dataBlock = getDataBlock(records2, header);
dataBlock = getDataBlock(DEFAULT_DATA_BLOCK_TYPE, records2, header);
writer.appendBlock(dataBlock);
writer.close();
@@ -1595,7 +1617,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
List<IndexedRecord> records3 = SchemaTestUtil.generateTestRecords(0, 100);
List<IndexedRecord> copyOfRecords3 = records3.stream()
.map(record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList());
dataBlock = getDataBlock(records3, header);
dataBlock = getDataBlock(DEFAULT_DATA_BLOCK_TYPE, records3, header);
writer.appendBlock(dataBlock);
writer.close();
@@ -1646,7 +1668,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
Map<HoodieLogBlock.HeaderMetadataType, String> header = new HashMap<>();
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
HoodieDataBlock dataBlock = getDataBlock(records, header);
HoodieDataBlock dataBlock = getDataBlock(DEFAULT_DATA_BLOCK_TYPE, records, header);
writer.appendBlock(dataBlock);
writer.close();
@@ -1674,7 +1696,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath).withFileExtension(HoodieLogFile.DELTA_EXTENSION)
.withFileId("test-fileid1").overBaseCommit("100").withFs(fs).build();
records = SchemaTestUtil.generateTestRecords(0, 100);
dataBlock = getDataBlock(records, header);
dataBlock = getDataBlock(DEFAULT_DATA_BLOCK_TYPE, records, header);
writer.appendBlock(dataBlock);
writer.close();
@@ -1708,7 +1730,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
Map<HoodieLogBlock.HeaderMetadataType, String> header = new HashMap<>();
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
HoodieDataBlock dataBlock = getDataBlock(records1, header);
HoodieDataBlock dataBlock = getDataBlock(DEFAULT_DATA_BLOCK_TYPE, records1, header);
writer.appendBlock(dataBlock);
writer.close();
@@ -1716,7 +1738,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath).withFileExtension(HoodieLogFile.DELTA_EXTENSION)
.withFileId("test-fileid1").overBaseCommit("100").withFs(fs).build();
List<IndexedRecord> records2 = SchemaTestUtil.generateTestRecords(0, 100);
dataBlock = getDataBlock(records2, header);
dataBlock = getDataBlock(DEFAULT_DATA_BLOCK_TYPE, records2, header);
writer.appendBlock(dataBlock);
writer.close();
@@ -1725,7 +1747,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath).withFileExtension(HoodieLogFile.DELTA_EXTENSION)
.withFileId("test-fileid1").overBaseCommit("100").withFs(fs).build();
List<IndexedRecord> records3 = SchemaTestUtil.generateTestRecords(0, 100);
dataBlock = getDataBlock(records3, header);
dataBlock = getDataBlock(DEFAULT_DATA_BLOCK_TYPE, records3, header);
writer.appendBlock(dataBlock);
writer.close();
@@ -1786,8 +1808,66 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
}
}
private HoodieDataBlock getDataBlock(List<IndexedRecord> records, Map<HeaderMetadataType, String> header) {
return getDataBlock(dataBlockType, records, header);
@ParameterizedTest
@EnumSource(names = {"AVRO_DATA_BLOCK", "HFILE_DATA_BLOCK", "PARQUET_DATA_BLOCK"})
public void testDataBlockFormatAppendAndReadWithProjectedSchema(
HoodieLogBlockType dataBlockType
) throws IOException, URISyntaxException, InterruptedException {
Writer writer = HoodieLogFormat.newWriterBuilder()
.onParentPath(partitionPath)
.withFileExtension(HoodieLogFile.DELTA_EXTENSION)
.withFileId("test-fileid1")
.overBaseCommit("100")
.withFs(fs)
.build();
List<GenericRecord> records = SchemaTestUtil.generateTestGenericRecords(0, 1000);
Schema schema = getSimpleSchema();
Map<HoodieLogBlock.HeaderMetadataType, String> header =
new HashMap<HoodieLogBlock.HeaderMetadataType, String>() {{
put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
}};
// Init Benchmark to report number of bytes actually read from the Block
BenchmarkCounter.initCounterFromReporter(HadoopMapRedUtils.createTestReporter(), fs.getConf());
// NOTE: Have to use this ugly hack since List generic is not covariant in its type param
HoodieDataBlock dataBlock = getDataBlock(dataBlockType, (List<IndexedRecord>)(List) records, header);
writer.appendBlock(dataBlock);
writer.close();
Schema projectedSchema = HoodieAvroUtils.generateProjectionSchema(schema, Collections.singletonList("name"));
List<GenericRecord> projectedRecords = HoodieAvroUtils.rewriteRecords(records, projectedSchema);
try (Reader reader = HoodieLogFormat.newReader(fs, writer.getLogFile(), projectedSchema, true, false)) {
assertTrue(reader.hasNext(), "First block should be available");
HoodieLogBlock nextBlock = reader.next();
HoodieDataBlock dataBlockRead = (HoodieDataBlock) nextBlock;
Map<HoodieLogBlockType, Integer> expectedReadBytes =
new HashMap<HoodieLogBlockType, Integer>() {{
put(HoodieLogBlockType.AVRO_DATA_BLOCK, 0); // not supported
put(HoodieLogBlockType.HFILE_DATA_BLOCK, 0); // not supported
put(HoodieLogBlockType.PARQUET_DATA_BLOCK, 2605);
}};
assertEquals(projectedRecords.size(), dataBlockRead.getRecords().size(),
"Read records size should be equal to the written records size");
assertEquals(projectedRecords, dataBlockRead.getRecords(),
"Both records lists should be the same. (ordering guaranteed)");
assertEquals(dataBlockRead.getSchema(), projectedSchema);
int bytesRead = (int) BenchmarkCounter.getBytesRead();
assertEquals(expectedReadBytes.get(dataBlockType), bytesRead, "Read bytes have to match");
}
}
private HoodieDataBlock getDataBlock(HoodieLogBlockType dataBlockType, List<IndexedRecord> records,
@@ -1796,7 +1876,9 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
case AVRO_DATA_BLOCK:
return new HoodieAvroDataBlock(records, header, HoodieRecord.RECORD_KEY_METADATA_FIELD);
case HFILE_DATA_BLOCK:
return new HoodieHFileDataBlock(records, header, HoodieRecord.RECORD_KEY_METADATA_FIELD);
return new HoodieHFileDataBlock(records, header, Compression.Algorithm.GZ);
case PARQUET_DATA_BLOCK:
return new HoodieParquetDataBlock(records, header, HoodieRecord.RECORD_KEY_METADATA_FIELD, CompressionCodecName.GZIP);
default:
throw new RuntimeException("Unknown data block type " + dataBlockType);
}

View File

@@ -19,6 +19,7 @@
package org.apache.hudi.common.functional;
import org.apache.hudi.common.model.HoodieArchivedLogFile;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.table.log.HoodieLogFormat;
import org.apache.hudi.common.table.log.HoodieLogFormat.Writer;
import org.apache.hudi.common.table.log.block.HoodieAvroDataBlock;
@@ -104,7 +105,7 @@ public class TestHoodieLogFormatAppendFailure {
Map<HoodieLogBlock.HeaderMetadataType, String> header = new HashMap<>(2);
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString());
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records, header);
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records, header, HoodieRecord.RECORD_KEY_METADATA_FIELD);
Writer writer = HoodieLogFormat.newWriterBuilder().onParentPath(testPath)
.withFileExtension(HoodieArchivedLogFile.ARCHIVE_EXTENSION).withFileId("commits.archive")

View File

@@ -0,0 +1,87 @@
/*
* 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.testutils;
import org.apache.hadoop.mapred.Counters;
import org.apache.hadoop.mapred.InputSplit;
import org.apache.hadoop.mapred.Reporter;
import org.apache.hudi.common.util.Option;
import java.util.concurrent.ConcurrentHashMap;
public class HadoopMapRedUtils {
/**
* Creates instance of {@link Reporter} to collect reported counters
*/
public static Reporter createTestReporter() {
class TestReporter implements Reporter {
private final ConcurrentHashMap<String, Counters.Counter> counters =
new ConcurrentHashMap<>();
@Override
public void setStatus(String status) {
// not-supported
}
@Override
public Counters.Counter getCounter(Enum<?> name) {
return counters.computeIfAbsent(name.name(), (ignored) -> new Counters.Counter());
}
@Override
public Counters.Counter getCounter(String group, String name) {
return counters.computeIfAbsent(getKey(group, name), (ignored) -> new Counters.Counter());
}
@Override
public void incrCounter(Enum<?> key, long amount) {
Option.ofNullable(counters.get(key))
.ifPresent(c -> c.increment(amount));
}
@Override
public void incrCounter(String group, String counter, long amount) {
Option.ofNullable(counters.get(getKey(group, counter)))
.ifPresent(c -> c.increment(amount));
}
@Override
public InputSplit getInputSplit() throws UnsupportedOperationException {
throw new UnsupportedOperationException("not supported");
}
@Override
public float getProgress() {
return -1;
}
@Override
public void progress() {
// not-supported
}
private String getKey(String group, String name) {
return String.format("%s:%s", group, name);
}
}
return new TestReporter();
}
}

View File

@@ -71,6 +71,10 @@ public final class SchemaTestUtil {
return toRecords(getSimpleSchema(), getSimpleSchema(), from, limit);
}
public static List<GenericRecord> generateTestGenericRecords(int from, int limit) throws IOException, URISyntaxException {
return toRecords(getSimpleSchema(), getSimpleSchema(), from, limit);
}
public static List<String> generateTestJsonRecords(int from, int limit) throws IOException, URISyntaxException {
Path dataPath = initializeSampleDataPath();
@@ -81,9 +85,9 @@ public final class SchemaTestUtil {
}
}
private static List<IndexedRecord> toRecords(Schema writerSchema, Schema readerSchema, int from, int limit)
private static <T extends IndexedRecord> List<T> toRecords(Schema writerSchema, Schema readerSchema, int from, int limit)
throws IOException, URISyntaxException {
GenericDatumReader<IndexedRecord> reader = new GenericDatumReader<>(writerSchema, readerSchema);
GenericDatumReader<T> reader = new GenericDatumReader<>(writerSchema, readerSchema);
Path dataPath = initializeSampleDataPath();
try (Stream<String> stream = Files.lines(dataPath)) {

View File

@@ -0,0 +1,113 @@
/*
* 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.io;
import org.junit.jupiter.api.Test;
import java.nio.ByteBuffer;
import static org.junit.jupiter.api.Assertions.assertArrayEquals;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertThrows;
public class TestByteBufferBackedInputStream {
@Test
public void testConstructor() {
byte[] bytes = { 0xD, 0xE, 0xA, 0xD, 0xD, 0xE, 0xE, 0xD };
ByteBuffer byteBuf = ByteBuffer.wrap(bytes, 0, 1);
ByteBuffer byteBufClone = byteBuf.duplicate();
// ByteBuffer ctor
ByteBufferBackedInputStream first = new ByteBufferBackedInputStream(byteBuf);
assertEquals(first.read(), 0xD);
assertThrows(IllegalArgumentException.class, first::read);
// Make sure that the original buffer stays intact
assertEquals(byteBufClone, byteBuf);
// byte[] ctor
ByteBufferBackedInputStream second = new ByteBufferBackedInputStream(bytes);
assertEquals(second.read(), 0xD);
// byte[] ctor (w/ offset)
ByteBufferBackedInputStream third = new ByteBufferBackedInputStream(bytes, 1, 1);
assertEquals(third.read(), 0xE);
assertThrows(IllegalArgumentException.class, third::read);
}
@Test
public void testRead() {
byte[] sourceBytes = { 0xD, 0xE, 0xA, 0xD, 0xD, 0xE, 0xE, 0xD };
ByteBufferBackedInputStream stream = new ByteBufferBackedInputStream(sourceBytes);
int firstByte = stream.read();
assertEquals(firstByte, 0xD);
byte[] readBytes = new byte[4];
int read = stream.read(readBytes, 1, 3);
assertEquals(3, read);
assertArrayEquals(new byte[]{0, 0xE, 0xA, 0xD}, readBytes);
assertEquals(4, stream.getPosition());
}
@Test
public void testSeek() {
byte[] sourceBytes = { 0xD, 0xE, 0xA, 0xD, 0xD, 0xA, 0xE, 0xD };
ByteBufferBackedInputStream stream = new ByteBufferBackedInputStream(sourceBytes, 1, 7);
// Seek to 2 byte in the stream (3 in the original buffer)
stream.seek(1);
int firstRead = stream.read();
assertEquals(0xA, firstRead);
// Seek to 5 byte in the stream (6 in the original buffer)
stream.seek(5);
int secondRead = stream.read();
assertEquals(0xE, secondRead);
// Try to seek past the stream boundary
assertThrows(IllegalArgumentException.class, () -> stream.seek(8));
}
@Test
public void testCopyFrom() {
byte[] sourceBytes = { 0xD, 0xE, 0xA, 0xD, 0xD, 0xA, 0xE, 0xD };
ByteBufferBackedInputStream stream = new ByteBufferBackedInputStream(sourceBytes);
int firstByte = stream.read();
assertEquals(firstByte, 0xD);
// Copy 5 byes from the stream (while keeping stream's state intact)
byte[] targetBytes = new byte[5];
stream.copyFrom(2, targetBytes, 0, targetBytes.length);
assertArrayEquals(new byte[] { 0xA, 0xD, 0xD, 0xA, 0xE }, targetBytes);
// Continue reading the stream from where we left of (before copying)
int secondByte = stream.read();
assertEquals(secondByte, 0xE);
}
}