1
0

[HUDI-684] Introduced abstraction for writing and reading different types of base file formats. (#1687)

Notable changes:
    1. HoodieFileWriter and HoodieFileReader abstractions for writer/reader side of a base file format
    2. HoodieDataBlock abstraction for creation specific data blocks for base file formats. (e.g. Parquet has HoodieAvroDataBlock)
    3. All hardocded references to Parquet / Parquet based classes have been abstracted to call methods which accept a base file format
    4. HiveSyncTool accepts the base file format as a CLI parameter
    5. HoodieDeltaStreamer accepts the base file format as a CLI parameter
    6. HoodieSparkSqlWriter accepts the base file format as a parameter
This commit is contained in:
Prashant Wason
2020-06-25 23:46:55 -07:00
committed by GitHub
parent 5e47673341
commit 2603cfb33e
55 changed files with 1086 additions and 466 deletions

View File

@@ -0,0 +1,30 @@
/*
* 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.functional;
import org.apache.hudi.common.table.log.block.HoodieLogBlock.HoodieLogBlockType;
/**
* Tests Avro log format {@link HoodieAvroDataBlock}.
*/
public class TestHoodieAvroLogFormat extends TestHoodieLogFormat {
public TestHoodieAvroLogFormat() {
super(HoodieLogBlockType.AVRO_DATA_BLOCK);
}
}

View File

@@ -32,6 +32,7 @@ import org.apache.hudi.common.table.log.HoodieLogFormat.Writer;
import org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner;
import org.apache.hudi.common.table.log.block.HoodieAvroDataBlock;
import org.apache.hudi.common.table.log.block.HoodieCommandBlock;
import org.apache.hudi.common.table.log.block.HoodieDataBlock;
import org.apache.hudi.common.table.log.block.HoodieDeleteBlock;
import org.apache.hudi.common.table.log.block.HoodieLogBlock;
import org.apache.hudi.common.table.log.block.HoodieLogBlock.HeaderMetadataType;
@@ -81,12 +82,20 @@ import static org.junit.jupiter.api.Assertions.assertTrue;
* Tests hoodie log format {@link HoodieLogFormat}.
*/
@SuppressWarnings("Duplicates")
public class TestHoodieLogFormat extends HoodieCommonTestHarness {
public abstract class TestHoodieLogFormat extends HoodieCommonTestHarness {
private static String BASE_OUTPUT_PATH = "/tmp/";
private FileSystem fs;
private Path partitionPath;
private int bufferSize = 4096;
private HoodieLogBlockType dataBlockType;
public TestHoodieLogFormat(HoodieLogBlockType dataBlockType) {
this.dataBlockType = dataBlockType;
}
private TestHoodieLogFormat() {
}
@BeforeAll
public static void setUpClass() throws IOException, InterruptedException {
@@ -133,7 +142,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
Map<HeaderMetadataType, String> header = new HashMap<>();
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString());
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records, header);
HoodieDataBlock dataBlock = getDataBlock(records, header);
writer = writer.appendBlock(dataBlock);
long size = writer.getCurrentSize();
assertTrue(size > 0, "We just wrote a block - size should be > 0");
@@ -151,7 +160,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());
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records, header);
HoodieDataBlock dataBlock = getDataBlock(records, header);
// Write out a block
writer = writer.appendBlock(dataBlock);
// Get the size of the block
@@ -164,7 +173,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
.withFileId("test-fileid1").overBaseCommit("100").withFs(fs).withSizeThreshold(size - 1).build();
records = SchemaTestUtil.generateTestRecords(0, 100);
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString());
dataBlock = new HoodieAvroDataBlock(records, header);
dataBlock = getDataBlock(records, header);
writer = writer.appendBlock(dataBlock);
assertEquals(0, writer.getCurrentSize(), "This should be a new log file and hence size should be 0");
assertEquals(2, writer.getLogFile().getLogVersion(), "Version should be rolled to 2");
@@ -217,7 +226,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());
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records, header);
HoodieDataBlock dataBlock = getDataBlock(records, header);
writer = writer.appendBlock(dataBlock);
writer2 = writer2.appendBlock(dataBlock);
writer.close();
@@ -235,7 +244,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());
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records, header);
HoodieDataBlock dataBlock = getDataBlock(records, header);
writer = writer.appendBlock(dataBlock);
long size1 = writer.getCurrentSize();
writer.close();
@@ -245,7 +254,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 = new HoodieAvroDataBlock(records, header);
dataBlock = getDataBlock(records, header);
writer = writer.appendBlock(dataBlock);
long size2 = writer.getCurrentSize();
assertTrue(size2 > size1, "We just wrote a new block - size2 should be > size1");
@@ -259,7 +268,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 = new HoodieAvroDataBlock(records, header);
dataBlock = getDataBlock(records, header);
writer = writer.appendBlock(dataBlock);
long size3 = writer.getCurrentSize();
assertTrue(size3 > size2, "We just wrote a new block - size3 should be > size2");
@@ -289,7 +298,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
* 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);
@@ -313,7 +322,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());
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records, header);
HoodieDataBlock dataBlock = getDataBlock(records, header);
for (int i = 0; i < 2; i++) {
HoodieLogFormat.newWriterBuilder().onParentPath(testPath)
@@ -338,15 +347,15 @@ 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());
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records, header);
HoodieDataBlock dataBlock = getDataBlock(records, header);
writer = 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(HoodieLogBlockType.AVRO_DATA_BLOCK, nextBlock.getBlockType(), "The next block should be a data block");
HoodieAvroDataBlock dataBlockRead = (HoodieAvroDataBlock) nextBlock;
assertEquals(dataBlockType, 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");
assertEquals(copyOfRecords, dataBlockRead.getRecords(),
@@ -366,7 +375,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());
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1, header);
HoodieDataBlock dataBlock = getDataBlock(records1, header);
writer = writer.appendBlock(dataBlock);
writer.close();
@@ -377,7 +386,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, getSimpleSchema().toString());
dataBlock = new HoodieAvroDataBlock(records2, header);
dataBlock = getDataBlock(records2, header);
writer = writer.appendBlock(dataBlock);
writer.close();
@@ -389,14 +398,14 @@ 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, getSimpleSchema().toString());
dataBlock = new HoodieAvroDataBlock(records3, header);
dataBlock = getDataBlock(records3, header);
writer = writer.appendBlock(dataBlock);
writer.close();
Reader reader = HoodieLogFormat.newReader(fs, writer.getLogFile(), SchemaTestUtil.getSimpleSchema());
assertTrue(reader.hasNext(), "First block should be available");
HoodieLogBlock nextBlock = reader.next();
HoodieAvroDataBlock dataBlockRead = (HoodieAvroDataBlock) nextBlock;
HoodieDataBlock dataBlockRead = (HoodieDataBlock) nextBlock;
assertEquals(copyOfRecords1.size(), dataBlockRead.getRecords().size(),
"Read records size should be equal to the written records size");
assertEquals(copyOfRecords1, dataBlockRead.getRecords(),
@@ -405,7 +414,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
reader.hasNext();
nextBlock = reader.next();
dataBlockRead = (HoodieAvroDataBlock) nextBlock;
dataBlockRead = (HoodieDataBlock) nextBlock;
assertEquals(copyOfRecords2.size(), dataBlockRead.getRecords().size(),
"Read records size should be equal to the written records size");
assertEquals(copyOfRecords2, dataBlockRead.getRecords(),
@@ -413,7 +422,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
reader.hasNext();
nextBlock = reader.next();
dataBlockRead = (HoodieAvroDataBlock) nextBlock;
dataBlockRead = (HoodieDataBlock) nextBlock;
assertEquals(copyOfRecords3.size(), dataBlockRead.getRecords().size(),
"Read records size should be equal to the written records size");
assertEquals(copyOfRecords3, dataBlockRead.getRecords(),
@@ -443,7 +452,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());
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1, header);
HoodieDataBlock dataBlock = getDataBlock(records1, header);
writer = writer.appendBlock(dataBlock);
}
writer.close();
@@ -472,7 +481,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());
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records, header);
HoodieDataBlock dataBlock = getDataBlock(records, header);
writer = writer.appendBlock(dataBlock);
writer.close();
@@ -498,7 +507,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 = new HoodieAvroDataBlock(records, header);
dataBlock = getDataBlock(records, header);
writer = writer.appendBlock(dataBlock);
writer.close();
@@ -536,7 +545,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 = new HoodieAvroDataBlock(records, header);
dataBlock = getDataBlock(records, header);
writer = writer.appendBlock(dataBlock);
writer.close();
@@ -574,7 +583,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());
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1, header);
HoodieDataBlock dataBlock = getDataBlock(records1, header);
writer = writer.appendBlock(dataBlock);
// Write 2
@@ -582,7 +591,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 = new HoodieAvroDataBlock(records2, header);
dataBlock = getDataBlock(records2, header);
writer = writer.appendBlock(dataBlock);
writer.close();
@@ -621,14 +630,14 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1, header);
HoodieDataBlock dataBlock = getDataBlock(records1, header);
writer = 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 = new HoodieAvroDataBlock(records2, header);
dataBlock = getDataBlock(records2, header);
writer = writer.appendBlock(dataBlock);
// Rollback the last write
@@ -644,7 +653,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 = new HoodieAvroDataBlock(records3, header);
dataBlock = getDataBlock(records3, header);
writer = writer.appendBlock(dataBlock);
writer.close();
@@ -681,7 +690,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());
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1, header);
HoodieDataBlock dataBlock = getDataBlock(records1, header);
writer = writer.appendBlock(dataBlock);
writer.close();
@@ -723,7 +732,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
.map(record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList());
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
dataBlock = new HoodieAvroDataBlock(records3, header);
dataBlock = getDataBlock(records3, header);
writer = writer.appendBlock(dataBlock);
writer.close();
@@ -761,7 +770,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());
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1, header);
HoodieDataBlock dataBlock = getDataBlock(records1, header);
writer = writer.appendBlock(dataBlock);
// Write 2
@@ -769,7 +778,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 = new HoodieAvroDataBlock(records2, header);
dataBlock = getDataBlock(records2, header);
writer = writer.appendBlock(dataBlock);
copyOfRecords1.addAll(copyOfRecords2);
@@ -849,13 +858,13 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
header.put(HoodieLogBlock.HeaderMetadataType.TARGET_INSTANT_TIME, "100");
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1, header);
HoodieDataBlock dataBlock = getDataBlock(records1, header);
writer = writer.appendBlock(dataBlock);
// Write 2
List<IndexedRecord> records2 = SchemaTestUtil.generateHoodieTestRecords(0, 100);
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
dataBlock = new HoodieAvroDataBlock(records2, header);
dataBlock = getDataBlock(records2, header);
writer = writer.appendBlock(dataBlock);
// Delete 50 keys
@@ -916,7 +925,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());
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1, header);
HoodieDataBlock dataBlock = getDataBlock(records1, header);
writer = writer.appendBlock(dataBlock);
// Delete 50 keys
@@ -958,7 +967,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());
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1, header);
HoodieDataBlock dataBlock = getDataBlock(records1, header);
writer = writer.appendBlock(dataBlock);
// Write invalid rollback for a failed write (possible for in-flight commits)
@@ -1000,7 +1009,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());
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1, header);
HoodieDataBlock dataBlock = getDataBlock(records1, header);
writer = writer.appendBlock(dataBlock);
writer = writer.appendBlock(dataBlock);
writer = writer.appendBlock(dataBlock);
@@ -1047,7 +1056,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());
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1, header);
HoodieDataBlock dataBlock = getDataBlock(records1, header);
writer = writer.appendBlock(dataBlock);
writer = writer.appendBlock(dataBlock);
writer = writer.appendBlock(dataBlock);
@@ -1149,7 +1158,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());
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records.subList(0, numRecordsInLog1), header);
HoodieDataBlock dataBlock = getDataBlock(records.subList(0, numRecordsInLog1), header);
writer = writer.appendBlock(dataBlock);
// Get the size of the block
long size = writer.getCurrentSize();
@@ -1163,7 +1172,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());
HoodieAvroDataBlock dataBlock2 = new HoodieAvroDataBlock(records2.subList(0, numRecordsInLog2), header2);
HoodieDataBlock dataBlock2 = getDataBlock(records2.subList(0, numRecordsInLog2), header2);
writer2 = writer2.appendBlock(dataBlock2);
// Get the size of the block
writer2.close();
@@ -1227,7 +1236,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());
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1, header);
HoodieDataBlock dataBlock = getDataBlock(records1, header);
writer = writer.appendBlock(dataBlock);
writer.close();
@@ -1237,7 +1246,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 = new HoodieAvroDataBlock(records2, header);
dataBlock = getDataBlock(records2, header);
writer = writer.appendBlock(dataBlock);
writer.close();
@@ -1248,7 +1257,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 = new HoodieAvroDataBlock(records3, header);
dataBlock = getDataBlock(records3, header);
writer = writer.appendBlock(dataBlock);
writer.close();
@@ -1257,7 +1266,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
assertTrue(reader.hasPrev(), "Last block should be available");
HoodieLogBlock prevBlock = reader.prev();
HoodieAvroDataBlock dataBlockRead = (HoodieAvroDataBlock) prevBlock;
HoodieDataBlock dataBlockRead = (HoodieDataBlock) prevBlock;
assertEquals(copyOfRecords3.size(), dataBlockRead.getRecords().size(),
"Third records size should be equal to the written records size");
@@ -1266,7 +1275,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
assertTrue(reader.hasPrev(), "Second block should be available");
prevBlock = reader.prev();
dataBlockRead = (HoodieAvroDataBlock) prevBlock;
dataBlockRead = (HoodieDataBlock) prevBlock;
assertEquals(copyOfRecords2.size(), dataBlockRead.getRecords().size(),
"Read records size should be equal to the written records size");
assertEquals(copyOfRecords2, dataBlockRead.getRecords(),
@@ -1274,7 +1283,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
assertTrue(reader.hasPrev(), "First block should be available");
prevBlock = reader.prev();
dataBlockRead = (HoodieAvroDataBlock) prevBlock;
dataBlockRead = (HoodieDataBlock) prevBlock;
assertEquals(copyOfRecords1.size(), dataBlockRead.getRecords().size(),
"Read records size should be equal to the written records size");
assertEquals(copyOfRecords1, dataBlockRead.getRecords(),
@@ -1296,7 +1305,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());
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records, header);
HoodieDataBlock dataBlock = getDataBlock(records, header);
writer = writer.appendBlock(dataBlock);
writer.close();
@@ -1322,7 +1331,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 = new HoodieAvroDataBlock(records, header);
dataBlock = getDataBlock(records, header);
writer = writer.appendBlock(dataBlock);
writer.close();
@@ -1332,7 +1341,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
assertTrue(reader.hasPrev(), "Last block should be available");
HoodieLogBlock block = reader.prev();
assertTrue(block instanceof HoodieAvroDataBlock, "Last block should be datablock");
assertTrue(block instanceof HoodieDataBlock, "Last block should be datablock");
assertTrue(reader.hasPrev(), "Last block should be available");
assertThrows(CorruptedLogFileException.class, () -> {
@@ -1355,7 +1364,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());
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1, header);
HoodieDataBlock dataBlock = getDataBlock(records1, header);
writer = writer.appendBlock(dataBlock);
writer.close();
@@ -1363,7 +1372,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 = new HoodieAvroDataBlock(records2, header);
dataBlock = getDataBlock(records2, header);
writer = writer.appendBlock(dataBlock);
writer.close();
@@ -1372,7 +1381,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 = new HoodieAvroDataBlock(records3, header);
dataBlock = getDataBlock(records3, header);
writer = writer.appendBlock(dataBlock);
writer.close();
@@ -1388,7 +1397,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
// After moving twice, this last reader.prev() should read the First block written
assertTrue(reader.hasPrev(), "First block should be available");
HoodieLogBlock prevBlock = reader.prev();
HoodieAvroDataBlock dataBlockRead = (HoodieAvroDataBlock) prevBlock;
HoodieDataBlock dataBlockRead = (HoodieDataBlock) prevBlock;
assertEquals(copyOfRecords1.size(), dataBlockRead.getRecords().size(),
"Read records size should be equal to the written records size");
assertEquals(copyOfRecords1, dataBlockRead.getRecords(),
@@ -1429,4 +1438,13 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
assertEquals(recordsCopy.get(i), readRecords.get(i));
}
}
private HoodieDataBlock getDataBlock(List<IndexedRecord> records, Map<HeaderMetadataType, String> header) {
switch (dataBlockType) {
case AVRO_DATA_BLOCK:
return new HoodieAvroDataBlock(records, header);
default:
throw new RuntimeException("Unknown data block type " + dataBlockType);
}
}
}

View File

@@ -28,6 +28,7 @@ import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.FileSlice;
import org.apache.hudi.common.model.HoodieAvroPayload;
import org.apache.hudi.common.model.HoodieCleaningPolicy;
import org.apache.hudi.common.model.HoodieFileFormat;
import org.apache.hudi.common.model.HoodieLogFile;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecordLocation;
@@ -120,21 +121,41 @@ public class HoodieTestUtils {
return init(getDefaultHadoopConf(), basePath, tableType);
}
public static HoodieTableMetaClient init(String basePath, HoodieFileFormat baseFileFormat) throws IOException {
return init(getDefaultHadoopConf(), basePath, HoodieTableType.COPY_ON_WRITE, baseFileFormat);
}
public static HoodieTableMetaClient init(Configuration hadoopConf, String basePath) throws IOException {
return init(hadoopConf, basePath, HoodieTableType.COPY_ON_WRITE);
}
public static HoodieTableMetaClient init(Configuration hadoopConf, String basePath, HoodieTableType tableType)
throws IOException {
return init(hadoopConf, basePath, tableType, RAW_TRIPS_TEST_NAME);
return init(hadoopConf, basePath, tableType, new Properties());
}
public static HoodieTableMetaClient init(Configuration hadoopConf, String basePath, HoodieTableType tableType,
String tableName) throws IOException {
String tableName)
throws IOException {
Properties properties = new Properties();
properties.setProperty(HoodieTableConfig.HOODIE_TABLE_NAME_PROP_NAME, tableName);
properties.setProperty(HoodieTableConfig.HOODIE_TABLE_TYPE_PROP_NAME, tableType.name());
properties.setProperty(HoodieTableConfig.HOODIE_PAYLOAD_CLASS_PROP_NAME, HoodieAvroPayload.class.getName());
return init(hadoopConf, basePath, tableType, properties);
}
public static HoodieTableMetaClient init(Configuration hadoopConf, String basePath, HoodieTableType tableType,
HoodieFileFormat baseFileFormat)
throws IOException {
Properties properties = new Properties();
properties.setProperty(HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP_NAME, baseFileFormat.toString());
return init(hadoopConf, basePath, tableType, properties);
}
public static HoodieTableMetaClient init(Configuration hadoopConf, String basePath, HoodieTableType tableType,
Properties properties)
throws IOException {
properties.putIfAbsent(HoodieTableConfig.HOODIE_TABLE_NAME_PROP_NAME, RAW_TRIPS_TEST_NAME);
properties.putIfAbsent(HoodieTableConfig.HOODIE_TABLE_TYPE_PROP_NAME, tableType.name());
properties.putIfAbsent(HoodieTableConfig.HOODIE_PAYLOAD_CLASS_PROP_NAME, HoodieAvroPayload.class.getName());
return HoodieTableMetaClient.initTableAndGetMetaClient(hadoopConf, basePath, properties);
}
@@ -442,12 +463,31 @@ public class HoodieTestUtils {
});
}
// TODO: should be removed
public static FileStatus[] listAllDataFilesInPath(FileSystem fs, String basePath) throws IOException {
return listAllDataFilesInPath(fs, basePath, ".parquet");
}
public static FileStatus[] listAllDataFilesInPath(FileSystem fs, String basePath, String datafileExtension)
throws IOException {
RemoteIterator<LocatedFileStatus> itr = fs.listFiles(new Path(basePath), true);
List<FileStatus> returns = new ArrayList<>();
while (itr.hasNext()) {
LocatedFileStatus status = itr.next();
if (status.getPath().getName().contains(".parquet")) {
if (status.getPath().getName().contains(datafileExtension)) {
returns.add(status);
}
}
return returns.toArray(new FileStatus[returns.size()]);
}
public static FileStatus[] listAllLogFilesInPath(FileSystem fs, String basePath, String logfileExtension)
throws IOException {
RemoteIterator<LocatedFileStatus> itr = fs.listFiles(new Path(basePath), true);
List<FileStatus> returns = new ArrayList<>();
while (itr.hasNext()) {
LocatedFileStatus status = itr.next();
if (status.getPath().getName().contains(logfileExtension)) {
returns.add(status);
}
}

View File

@@ -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;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.parquet.hadoop.ParquetReader;
import org.junit.jupiter.api.Test;
import java.io.IOException;
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.junit.jupiter.api.Assertions.assertTrue;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
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) {
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
assertEquals(1, iterator.next());
// no more entries to iterate on
assertFalse(iterator.hasNext());
assertThrows(HoodieIOException.class, iterator::next, "should throw an exception since there is only 1 record");
}
}

View File

@@ -0,0 +1,54 @@
/*
* 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.io.storage;
import org.apache.avro.generic.IndexedRecord;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.io.TempDir;
import java.io.IOException;
import static org.junit.jupiter.api.Assertions.assertThrows;
import static org.junit.jupiter.api.Assertions.assertTrue;
/**
* Tests for {@link HoodieFileReaderFactory}.
*/
public class TestHoodieFileReaderFactory {
@TempDir
public java.nio.file.Path tempDir;
@Test
public void testGetFileReader() throws IOException {
// parquet file format.
final Configuration hadoopConf = new Configuration();
final Path parquetPath = new Path("/partition/path/f1_1-0-1_000.parquet");
HoodieFileReader<IndexedRecord> parquetReader = HoodieFileReaderFactory.getFileReader(hadoopConf, parquetPath);
assertTrue(parquetReader instanceof HoodieParquetReader);
// other file format exception.
final Path logPath = new Path("/partition/path/f.b51192a8-574b-4a85-b246-bcfec03ac8bf_100.log.2_1-0-1");
final Throwable thrown = assertThrows(UnsupportedOperationException.class, () -> {
HoodieFileReader<IndexedRecord> logWriter = HoodieFileReaderFactory.getFileReader(hadoopConf, logPath);
}, "should fail since log storage reader is not supported yet.");
assertTrue(thrown.getMessage().contains("format not supported yet."));
}
}