1
0

[HUDI-371] Supporting hive combine input format for realtime tables (#1503)

This commit is contained in:
n3nash
2020-04-20 20:40:06 -07:00
committed by GitHub
parent 2a2f31d919
commit 332072bc6d
15 changed files with 1042 additions and 369 deletions

View File

@@ -20,14 +20,23 @@ package org.apache.hudi.hadoop;
import org.apache.hudi.avro.HoodieAvroUtils;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.HoodieLogFile;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieTestUtils;
import org.apache.hudi.common.table.log.HoodieLogFormat;
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.HoodieLogBlock;
import org.apache.hudi.common.util.SchemaTestUtil;
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.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
import org.apache.hadoop.hive.serde2.ColumnProjectionUtils;
import org.apache.hadoop.mapred.JobConf;
import org.apache.parquet.avro.AvroParquetWriter;
import org.junit.rules.TemporaryFolder;
@@ -37,9 +46,12 @@ import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.UUID;
import java.util.stream.Collectors;
public class InputFormatTestUtil {
@@ -202,4 +214,92 @@ public class InputFormatTestUtil {
}
}
public static HoodieLogFormat.Writer writeRollback(File partitionDir, FileSystem fs, String fileId, String baseCommit,
String newCommit, String rolledBackInstant, int logVersion)
throws InterruptedException, IOException {
HoodieLogFormat.Writer writer = HoodieLogFormat.newWriterBuilder().onParentPath(new Path(partitionDir.getPath())).withFileId(fileId)
.overBaseCommit(baseCommit).withFs(fs).withLogVersion(logVersion).withLogWriteToken("1-0-1")
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).build();
// generate metadata
Map<HoodieLogBlock.HeaderMetadataType, String> header = new HashMap<>();
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, newCommit);
header.put(HoodieLogBlock.HeaderMetadataType.TARGET_INSTANT_TIME, rolledBackInstant);
header.put(HoodieLogBlock.HeaderMetadataType.COMMAND_BLOCK_TYPE,
String.valueOf(HoodieCommandBlock.HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK.ordinal()));
// if update belongs to an existing log file
writer = writer.appendBlock(new HoodieCommandBlock(header));
return writer;
}
public static HoodieLogFormat.Writer writeDataBlockToLogFile(File partitionDir, FileSystem fs, Schema schema, String
fileId,
String baseCommit, String newCommit, int numberOfRecords, int offset, int logVersion)
throws InterruptedException, IOException {
HoodieLogFormat.Writer writer = HoodieLogFormat.newWriterBuilder().onParentPath(new Path(partitionDir.getPath()))
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId(fileId).withLogVersion(logVersion)
.withLogWriteToken("1-0-1").overBaseCommit(baseCommit).withFs(fs).build();
List<IndexedRecord> records = new ArrayList<>();
for (int i = offset; i < offset + numberOfRecords; i++) {
records.add(SchemaTestUtil.generateAvroRecordFromJson(schema, i, newCommit, "fileid0"));
}
Schema writeSchema = records.get(0).getSchema();
Map<HoodieLogBlock.HeaderMetadataType, String> header = new HashMap<>();
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, newCommit);
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, writeSchema.toString());
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records, header);
writer = writer.appendBlock(dataBlock);
return writer;
}
public static HoodieLogFormat.Writer writeRollbackBlockToLogFile(File partitionDir, FileSystem fs, Schema schema,
String
fileId, String baseCommit, String newCommit, String oldCommit, int logVersion)
throws InterruptedException, IOException {
HoodieLogFormat.Writer writer = HoodieLogFormat.newWriterBuilder().onParentPath(new Path(partitionDir.getPath()))
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId(fileId).overBaseCommit(baseCommit)
.withLogVersion(logVersion).withFs(fs).build();
Map<HoodieLogBlock.HeaderMetadataType, String> header = new HashMap<>();
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, newCommit);
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
header.put(HoodieLogBlock.HeaderMetadataType.TARGET_INSTANT_TIME, oldCommit);
header.put(HoodieLogBlock.HeaderMetadataType.COMMAND_BLOCK_TYPE,
String.valueOf(HoodieCommandBlock.HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK.ordinal()));
HoodieCommandBlock rollbackBlock = new HoodieCommandBlock(header);
writer = writer.appendBlock(rollbackBlock);
return writer;
}
public static void setPropsForInputFormat(JobConf jobConf,
Schema schema, String hiveColumnTypes) {
List<Schema.Field> fields = schema.getFields();
String names = fields.stream().map(f -> f.name().toString()).collect(Collectors.joining(","));
String postions = fields.stream().map(f -> String.valueOf(f.pos())).collect(Collectors.joining(","));
Configuration conf = HoodieTestUtils.getDefaultHadoopConf();
String hiveColumnNames = fields.stream().filter(field -> !field.name().equalsIgnoreCase("datestr"))
.map(Schema.Field::name).collect(Collectors.joining(","));
hiveColumnNames = hiveColumnNames + ",datestr";
String modifiedHiveColumnTypes = HoodieAvroUtils.addMetadataColumnTypes(hiveColumnTypes);
modifiedHiveColumnTypes = modifiedHiveColumnTypes + ",string";
jobConf.set(hive_metastoreConstants.META_TABLE_COLUMNS, hiveColumnNames);
jobConf.set(hive_metastoreConstants.META_TABLE_COLUMN_TYPES, modifiedHiveColumnTypes);
jobConf.set(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, names);
jobConf.set(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR, postions);
jobConf.set(hive_metastoreConstants.META_TABLE_PARTITION_COLUMNS, "datestr");
conf.set(hive_metastoreConstants.META_TABLE_COLUMNS, hiveColumnNames);
conf.set(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, names);
conf.set(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR, postions);
conf.set(hive_metastoreConstants.META_TABLE_PARTITION_COLUMNS, "datestr");
conf.set(hive_metastoreConstants.META_TABLE_COLUMN_TYPES, modifiedHiveColumnTypes);
jobConf.addResource(conf);
}
public static void setInputPath(JobConf jobConf, String inputPath) {
jobConf.set("mapreduce.input.fileinputformat.inputdir", inputPath);
jobConf.set("mapreduce.input.fileinputformat.inputdir", inputPath);
jobConf.set("map.input.dir", inputPath);
}
}

View File

@@ -56,9 +56,23 @@ import static org.junit.Assert.assertTrue;
public class TestHoodieParquetInputFormat {
@Rule
public TemporaryFolder basePath = new TemporaryFolder();
private HoodieParquetInputFormat inputFormat;
private JobConf jobConf;
public static void ensureFilesInCommit(String msg, FileStatus[] files, String commit, int expected) {
int count = 0;
for (FileStatus file : files) {
String commitTs = FSUtils.getCommitTime(file.getPath().getName());
if (commit.equals(commitTs)) {
count++;
}
}
assertEquals(msg, expected, count);
}
@Before
public void setUp() {
inputFormat = new HoodieParquetInputFormat();
@@ -66,9 +80,6 @@ public class TestHoodieParquetInputFormat {
inputFormat.setConf(jobConf);
}
@Rule
public TemporaryFolder basePath = new TemporaryFolder();
// Verify that HoodieParquetInputFormat does not return instants after pending compaction
@Test
public void testPendingCompactionWithActiveCommits() throws IOException {
@@ -373,15 +384,4 @@ public class TestHoodieParquetInputFormat {
assertEquals(msg, expectedNumberOfRecordsInCommit, actualCount);
assertEquals(msg, totalExpected, totalCount);
}
public static void ensureFilesInCommit(String msg, FileStatus[] files, String commit, int expected) {
int count = 0;
for (FileStatus file : files) {
String commitTs = FSUtils.getCommitTime(file.getPath().getName());
if (commit.equals(commitTs)) {
count++;
}
}
assertEquals(msg, expected, count);
}
}

View File

@@ -0,0 +1,160 @@
/*
* 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.hadoop.realtime;
import static org.apache.hadoop.hive.ql.exec.Utilities.HAS_MAP_WORK;
import static org.apache.hadoop.hive.ql.exec.Utilities.MAPRED_MAPPER_CLASS;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import org.apache.hudi.avro.HoodieAvroUtils;
import org.apache.hudi.common.HoodieCommonTestHarness;
import org.apache.hudi.common.minicluster.MiniClusterUtil;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.model.HoodieTestUtils;
import org.apache.hudi.common.table.log.HoodieLogFormat;
import org.apache.hudi.common.util.SchemaTestUtil;
import org.apache.hudi.hadoop.InputFormatTestUtil;
import org.apache.avro.Schema;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hive.ql.exec.Utilities;
import org.apache.hadoop.hive.ql.exec.mr.ExecMapper;
import org.apache.hadoop.hive.ql.plan.MapredWork;
import org.apache.hadoop.hive.ql.plan.PartitionDesc;
import org.apache.hadoop.hive.ql.plan.TableDesc;
import org.apache.hadoop.io.ArrayWritable;
import org.apache.hadoop.io.NullWritable;
import org.apache.hadoop.mapred.FileInputFormat;
import org.apache.hadoop.mapred.InputSplit;
import org.apache.hadoop.mapred.JobConf;
import org.apache.hadoop.mapred.RecordReader;
import org.apache.hudi.hadoop.hive.HoodieCombineHiveInputFormat;
import org.junit.AfterClass;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Ignore;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.TemporaryFolder;
import java.io.File;
import java.io.IOException;
import java.util.LinkedHashMap;
public class TestHoodieCombineHiveInputFormat extends HoodieCommonTestHarness {
@Rule
public TemporaryFolder basePath = new TemporaryFolder();
private JobConf jobConf;
private FileSystem fs;
private Configuration hadoopConf;
@BeforeClass
public static void setUpClass() throws IOException, InterruptedException {
// Append is not supported in LocalFileSystem. HDFS needs to be setup.
MiniClusterUtil.setUp();
}
@AfterClass
public static void tearDownClass() {
MiniClusterUtil.shutdown();
}
@Before
public void setUp() throws IOException, InterruptedException {
this.fs = MiniClusterUtil.fileSystem;
jobConf = new JobConf();
hadoopConf = HoodieTestUtils.getDefaultHadoopConf();
assertTrue(fs.mkdirs(new Path(folder.getRoot().getPath())));
HoodieTestUtils.init(MiniClusterUtil.configuration, basePath.getRoot().getPath(), HoodieTableType.MERGE_ON_READ);
}
@Test
@Ignore
public void testHoodieRealtimeCombineHoodieInputFormat() throws Exception {
Configuration conf = new Configuration();
// initial commit
Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getEvolvedSchema());
HoodieTestUtils.init(hadoopConf, basePath.getRoot().getAbsolutePath(), HoodieTableType.MERGE_ON_READ);
String commitTime = "100";
final int numRecords = 1000;
// Create 3 parquet files with 1000 records each
File partitionDir = InputFormatTestUtil.prepareParquetTable(basePath, schema, 3, numRecords, commitTime);
InputFormatTestUtil.commit(basePath, commitTime);
// insert 1000 update records to log file 0
String newCommitTime = "101";
HoodieLogFormat.Writer writer =
InputFormatTestUtil.writeDataBlockToLogFile(partitionDir, fs, schema, "fileid0", commitTime, newCommitTime,
numRecords, numRecords, 0);
writer.close();
// insert 1000 update records to log file 1
writer =
InputFormatTestUtil.writeDataBlockToLogFile(partitionDir, fs, schema, "fileid1", commitTime, newCommitTime,
numRecords, numRecords, 0);
writer.close();
// insert 1000 update records to log file 2
writer =
InputFormatTestUtil.writeDataBlockToLogFile(partitionDir, fs, schema, "fileid2", commitTime, newCommitTime,
numRecords, numRecords, 0);
writer.close();
TableDesc tblDesc = Utilities.defaultTd;
// Set the input format
tblDesc.setInputFileFormatClass(HoodieCombineHiveInputFormat.class);
PartitionDesc partDesc = new PartitionDesc(tblDesc, null);
LinkedHashMap<Path, PartitionDesc> pt = new LinkedHashMap<>();
pt.put(new Path(basePath.getRoot().getAbsolutePath()), partDesc);
MapredWork mrwork = new MapredWork();
mrwork.getMapWork().setPathToPartitionInfo(pt);
Path mapWorkPath = new Path(basePath.getRoot().getAbsolutePath());
Utilities.setMapRedWork(conf, mrwork, mapWorkPath);
jobConf = new JobConf(conf);
// Add the paths
FileInputFormat.setInputPaths(jobConf, partitionDir.getPath());
jobConf.set(HAS_MAP_WORK, "true");
// The following config tells Hive to choose ExecMapper to read the MAP_WORK
jobConf.set(MAPRED_MAPPER_CLASS, ExecMapper.class.getName());
// setting the split size to be 3 to create one split for 3 file groups
jobConf.set(org.apache.hadoop.mapreduce.lib.input.FileInputFormat.SPLIT_MAXSIZE, "3");
HoodieCombineHiveInputFormat combineHiveInputFormat = new HoodieCombineHiveInputFormat();
String tripsHiveColumnTypes = "double,string,string,string,double,double,double,double,double";
InputFormatTestUtil.setPropsForInputFormat(jobConf, schema, tripsHiveColumnTypes);
InputSplit[] splits = combineHiveInputFormat.getSplits(jobConf, 1);
// Since the SPLIT_SIZE is 3, we should create only 1 split with all 3 file groups
assertEquals(splits.length, 1);
RecordReader<NullWritable, ArrayWritable> recordReader =
combineHiveInputFormat.getRecordReader(splits[0], jobConf, null);
NullWritable nullWritable = recordReader.createKey();
ArrayWritable arrayWritable = recordReader.createValue();
int counter = 0;
while (recordReader.next(nullWritable, arrayWritable)) {
// read over all the splits
counter++;
}
// should read out 3 splits, each for file0, file1, file2 containing 1000 records each
assertEquals(3000, counter);
}
}

View File

@@ -26,11 +26,6 @@ import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.model.HoodieTestUtils;
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;
import org.apache.hudi.common.table.log.block.HoodieCommandBlock;
import org.apache.hudi.common.table.log.block.HoodieCommandBlock.HoodieCommandBlockTypeEnum;
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.timeline.HoodieTimeline;
import org.apache.hudi.common.util.SchemaTestUtil;
import org.apache.hudi.common.util.collection.Pair;
@@ -39,7 +34,6 @@ import org.apache.hudi.hadoop.InputFormatTestUtil;
import org.apache.avro.Schema;
import org.apache.avro.Schema.Field;
import org.apache.avro.generic.IndexedRecord;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
@@ -68,10 +62,8 @@ import java.io.File;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.stream.Collectors;
@@ -83,7 +75,8 @@ import static org.junit.Assert.assertTrue;
public class TestHoodieRealtimeRecordReader {
private static final String PARTITION_COLUMN = "datestr";
@Rule
public TemporaryFolder basePath = new TemporaryFolder();
private JobConf jobConf;
private FileSystem fs;
private Configuration hadoopConf;
@@ -96,64 +89,12 @@ public class TestHoodieRealtimeRecordReader {
fs = FSUtils.getFs(basePath.getRoot().getAbsolutePath(), hadoopConf);
}
@Rule
public TemporaryFolder basePath = new TemporaryFolder();
private Writer writeLogFile(File partitionDir, Schema schema, String fileId, String baseCommit, String newCommit,
private Writer writeLogFile(File partitionDir, Schema schema, String fileId, String baseCommit, String
newCommit,
int numberOfRecords) throws InterruptedException, IOException {
return writeDataBlockToLogFile(partitionDir, schema, fileId, baseCommit, newCommit, numberOfRecords, 0, 0);
}
private Writer writeRollback(File partitionDir, Schema schema, String fileId, String baseCommit, String newCommit,
String rolledBackInstant, int logVersion) throws InterruptedException, IOException {
Writer writer = HoodieLogFormat.newWriterBuilder().onParentPath(new Path(partitionDir.getPath())).withFileId(fileId)
.overBaseCommit(baseCommit).withFs(fs).withLogVersion(logVersion).withLogWriteToken("1-0-1")
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).build();
// generate metadata
Map<HeaderMetadataType, String> header = new HashMap<>();
header.put(HeaderMetadataType.INSTANT_TIME, newCommit);
header.put(HeaderMetadataType.TARGET_INSTANT_TIME, rolledBackInstant);
header.put(HeaderMetadataType.COMMAND_BLOCK_TYPE,
String.valueOf(HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK.ordinal()));
// if update belongs to an existing log file
writer = writer.appendBlock(new HoodieCommandBlock(header));
return writer;
}
private HoodieLogFormat.Writer writeDataBlockToLogFile(File partitionDir, Schema schema, String fileId,
String baseCommit, String newCommit, int numberOfRecords, int offset, int logVersion)
throws InterruptedException, IOException {
HoodieLogFormat.Writer writer = HoodieLogFormat.newWriterBuilder().onParentPath(new Path(partitionDir.getPath()))
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId(fileId).withLogVersion(logVersion)
.withLogWriteToken("1-0-1").overBaseCommit(baseCommit).withFs(fs).build();
List<IndexedRecord> records = new ArrayList<>();
for (int i = offset; i < offset + numberOfRecords; i++) {
records.add(SchemaTestUtil.generateAvroRecordFromJson(schema, i, newCommit, "fileid0"));
}
Schema writeSchema = records.get(0).getSchema();
Map<HeaderMetadataType, String> header = new HashMap<>();
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, newCommit);
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, writeSchema.toString());
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records, header);
writer = writer.appendBlock(dataBlock);
return writer;
}
private HoodieLogFormat.Writer writeRollbackBlockToLogFile(File partitionDir, Schema schema, String fileId,
String baseCommit, String newCommit, String oldCommit, int logVersion) throws InterruptedException, IOException {
HoodieLogFormat.Writer writer = HoodieLogFormat.newWriterBuilder().onParentPath(new Path(partitionDir.getPath()))
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId(fileId).overBaseCommit(baseCommit)
.withLogVersion(logVersion).withFs(fs).build();
Map<HoodieLogBlock.HeaderMetadataType, String> header = new HashMap<>();
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, newCommit);
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
header.put(HoodieLogBlock.HeaderMetadataType.TARGET_INSTANT_TIME, oldCommit);
header.put(HeaderMetadataType.COMMAND_BLOCK_TYPE,
String.valueOf(HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK.ordinal()));
HoodieCommandBlock rollbackBlock = new HoodieCommandBlock(header);
writer = writer.appendBlock(rollbackBlock);
return writer;
return InputFormatTestUtil.writeDataBlockToLogFile(partitionDir, fs, schema, fileId, baseCommit, newCommit,
numberOfRecords, 0,
0);
}
@Test
@@ -213,11 +154,12 @@ public class TestHoodieRealtimeRecordReader {
HoodieLogFormat.Writer writer;
if (action.equals(HoodieTimeline.ROLLBACK_ACTION)) {
writer = writeRollback(partitionDir, schema, "fileid0", baseInstant, instantTime,
writer = InputFormatTestUtil.writeRollback(partitionDir, fs, "fileid0", baseInstant, instantTime,
String.valueOf(baseInstantTs + logVersion - 1), logVersion);
} else {
writer =
writeDataBlockToLogFile(partitionDir, schema, "fileid0", baseInstant, instantTime, 100, 0, logVersion);
InputFormatTestUtil.writeDataBlockToLogFile(partitionDir, fs, schema, "fileid0", baseInstant,
instantTime, 100, 0, logVersion);
}
long size = writer.getCurrentSize();
writer.close();
@@ -228,7 +170,7 @@ public class TestHoodieRealtimeRecordReader {
HoodieRealtimeFileSplit split = new HoodieRealtimeFileSplit(
new FileSplit(new Path(partitionDir + "/fileid0_1-0-1_" + baseInstant + ".parquet"), 0, 1, jobConf),
basePath.getRoot().getPath(), fileSlice.getLogFiles().sorted(HoodieLogFile.getLogFileComparator())
.map(h -> h.getPath().toString()).collect(Collectors.toList()),
.map(h -> h.getPath().toString()).collect(Collectors.toList()),
instantTime);
// create a RecordReader to be used by HoodieRealtimeRecordReader
@@ -281,7 +223,8 @@ public class TestHoodieRealtimeRecordReader {
// insert new records to log file
String newCommitTime = "101";
HoodieLogFormat.Writer writer =
writeDataBlockToLogFile(partitionDir, schema, "fileid0", instantTime, newCommitTime, numRecords, numRecords, 0);
InputFormatTestUtil.writeDataBlockToLogFile(partitionDir, fs, schema, "fileid0", instantTime, newCommitTime,
numRecords, numRecords, 0);
long size = writer.getCurrentSize();
writer.close();
assertTrue("block - size should be > 0", size > 0);
@@ -302,7 +245,7 @@ public class TestHoodieRealtimeRecordReader {
jobConf.set(REALTIME_SKIP_MERGE_PROP, "true");
// validate unmerged record reader
HoodieRealtimeRecordReader recordReader = new HoodieRealtimeRecordReader(split, jobConf, reader);
RealtimeUnmergedRecordReader recordReader = new RealtimeUnmergedRecordReader(split, jobConf, reader);
// use reader to read base Parquet File and log file
// here all records should be present. Also ensure log records are in order.
@@ -486,7 +429,8 @@ public class TestHoodieRealtimeRecordReader {
schema = SchemaTestUtil.getComplexEvolvedSchema();
String newCommitTime = "101";
HoodieLogFormat.Writer writer =
writeDataBlockToLogFile(partitionDir, schema, "fileid0", instantTime, newCommitTime, numberOfLogRecords, 0, 1);
InputFormatTestUtil.writeDataBlockToLogFile(partitionDir, fs, schema, "fileid0", instantTime, newCommitTime,
numberOfLogRecords, 0, 1);
long size = writer.getCurrentSize();
logFilePaths.add(writer.getLogFile().getPath().toString());
writer.close();
@@ -494,7 +438,8 @@ public class TestHoodieRealtimeRecordReader {
// write rollback for the previous block in new log file version
newCommitTime = "102";
writer = writeRollbackBlockToLogFile(partitionDir, schema, "fileid0", instantTime, newCommitTime, "101", 1);
writer = InputFormatTestUtil.writeRollbackBlockToLogFile(partitionDir, fs, schema, "fileid0", instantTime,
newCommitTime, "101", 1);
logFilePaths.add(writer.getLogFile().getPath().toString());
writer.close();
InputFormatTestUtil.deltaCommit(basePath, newCommitTime);