1
0

[HUDI-2086]redo the logical of mor_incremental_view for hive (#3203)

This commit is contained in:
xiarixiaoyao
2021-11-10 15:41:07 +08:00
committed by GitHub
parent fd0f5df26d
commit a40ac62e0c
9 changed files with 1005 additions and 32 deletions

View File

@@ -18,17 +18,25 @@
package org.apache.hudi.hadoop.realtime;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.mapred.InputSplit;
import org.apache.hadoop.mapred.Reporter;
import org.apache.hudi.avro.HoodieAvroUtils;
import org.apache.hudi.avro.model.HoodieCompactionPlan;
import org.apache.hudi.common.config.HoodieCommonConfig;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.FileSlice;
import org.apache.hudi.common.model.HoodieCommitMetadata;
import org.apache.hudi.common.model.HoodieLogFile;
import org.apache.hudi.common.model.HoodieReplaceCommitMetadata;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.table.HoodieTableConfig;
import org.apache.hudi.common.model.HoodieWriteStat;
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.HoodieLogBlock;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.common.table.timeline.TimelineMetadataUtils;
import org.apache.hudi.common.testutils.FileCreateUtils;
import org.apache.hudi.common.testutils.HoodieTestUtils;
import org.apache.hudi.common.testutils.SchemaTestUtil;
@@ -36,8 +44,11 @@ import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.collection.ExternalSpillableMap;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.hadoop.config.HoodieRealtimeConfig;
import org.apache.hudi.hadoop.BaseFileWithLogsSplit;
import org.apache.hudi.hadoop.PathWithLogFilePath;
import org.apache.hudi.hadoop.RealtimeFileStatus;
import org.apache.hudi.hadoop.testutils.InputFormatTestUtil;
import org.apache.hudi.hadoop.config.HoodieRealtimeConfig;
import org.apache.avro.Schema;
import org.apache.avro.Schema.Field;
@@ -67,13 +78,17 @@ import org.junit.jupiter.params.provider.Arguments;
import org.junit.jupiter.params.provider.MethodSource;
import java.io.File;
import java.io.FileOutputStream;
import java.io.IOException;
import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Properties;
import java.util.Set;
import java.util.Map;
import java.util.HashMap;
import java.util.stream.Collectors;
import java.util.stream.Stream;
@@ -554,4 +569,256 @@ public class TestHoodieRealtimeRecordReader {
arguments(ExternalSpillableMap.DiskMapType.ROCKS_DB, true, true)
);
}
@Test
public void testIncrementalWithOnlylog() throws Exception {
// initial commit
Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getEvolvedSchema());
HoodieTestUtils.init(hadoopConf, basePath.toString(), HoodieTableType.MERGE_ON_READ);
String instantTime = "100";
final int numRecords = 1000;
File partitionDir = InputFormatTestUtil.prepareParquetTable(basePath, schema, 1, numRecords, instantTime,
HoodieTableType.MERGE_ON_READ);
//FileCreateUtils.createDeltaCommit(basePath.toString(), instantTime);
createDeltaCommitFile(basePath, instantTime,"2016/05/01", "2016/05/01/fileid0_1-0-1_100.parquet", "fileid0");
// Add the paths
FileInputFormat.setInputPaths(baseJobConf, partitionDir.getPath());
// insert new records to log file
try {
String newCommitTime = "102";
HoodieLogFormat.Writer writer =
InputFormatTestUtil.writeDataBlockToLogFile(partitionDir, fs, schema, "fileid0", instantTime, newCommitTime,
numRecords, numRecords, 0);
writer.close();
createDeltaCommitFile(basePath, newCommitTime,"2016/05/01", "2016/05/01/.fileid0_100.log.1_1-0-1", "fileid0");
InputFormatTestUtil.setupIncremental(baseJobConf, "101", 1);
HoodieParquetRealtimeInputFormat inputFormat = new HoodieParquetRealtimeInputFormat();
inputFormat.setConf(baseJobConf);
InputSplit[] splits = inputFormat.getSplits(baseJobConf, 1);
assertTrue(splits.length == 1);
JobConf newJobConf = new JobConf(baseJobConf);
List<Schema.Field> fields = schema.getFields();
setHiveColumnNameProps(fields, newJobConf, false);
RecordReader<NullWritable, ArrayWritable> reader = inputFormat.getRecordReader(splits[0], newJobConf, Reporter.NULL);
// use reader to read log file.
NullWritable key = reader.createKey();
ArrayWritable value = reader.createValue();
while (reader.next(key, value)) {
Writable[] values = value.get();
// since we set incremental start commit as 101 and commit_number as 1.
// the data belong to commit 102 should be read out.
assertEquals(newCommitTime, values[0].toString());
key = reader.createKey();
value = reader.createValue();
}
reader.close();
} catch (IOException e) {
throw new HoodieException(e.getMessage(), e);
}
}
@Test
public void testIncrementalWithReplace() throws Exception {
// initial commit
Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getEvolvedSchema());
HoodieTestUtils.init(hadoopConf, basePath.toString(), HoodieTableType.MERGE_ON_READ);
String baseInstant = "100";
File partitionDir = InputFormatTestUtil.prepareParquetTable(basePath, schema, 1, 100, baseInstant,
HoodieTableType.MERGE_ON_READ);
//FileCreateUtils.createDeltaCommit(basePath.toString(), instantTime);
createDeltaCommitFile(basePath, baseInstant,"2016/05/01", "2016/05/01/fileid0_1-0-1_100.parquet", "fileid0");
// Add the paths
FileInputFormat.setInputPaths(baseJobConf, partitionDir.getPath());
InputFormatTestUtil.simulateInserts(partitionDir, ".parquet", "fileid1", 1, "200");
Map<String, List<String>> partitionToReplaceFileIds = new HashMap<>();
List<String> replacedFileId = new ArrayList<>();
replacedFileId.add("fileid0");
partitionToReplaceFileIds.put("2016/05/01", replacedFileId);
createReplaceCommitFile(basePath,
"200","2016/05/01", "2016/05/01/fileid10_1-0-1_200.parquet", "fileid10", partitionToReplaceFileIds);
InputFormatTestUtil.setupIncremental(baseJobConf, "0", 1);
HoodieParquetRealtimeInputFormat inputFormat = new HoodieParquetRealtimeInputFormat();
inputFormat.setConf(baseJobConf);
InputSplit[] splits = inputFormat.getSplits(baseJobConf, 1);
assertTrue(splits.length == 1);
JobConf newJobConf = new JobConf(baseJobConf);
List<Schema.Field> fields = schema.getFields();
setHiveColumnNameProps(fields, newJobConf, false);
newJobConf.set("columns.types", "string,string,string,string,string,string,string,string,bigint,string,string");
RecordReader<NullWritable, ArrayWritable> reader = inputFormat.getRecordReader(splits[0], newJobConf, Reporter.NULL);
// use reader to read log file.
NullWritable key = reader.createKey();
ArrayWritable value = reader.createValue();
while (reader.next(key, value)) {
Writable[] values = value.get();
// since we set incremental start commit as 0 and commit_number as 1.
// the data belong to commit 100 should be read out.
assertEquals("100", values[0].toString());
key = reader.createKey();
value = reader.createValue();
}
reader.close();
}
private void createReplaceCommitFile(
java.nio.file.Path basePath,
String commitNumber,
String partitionPath,
String filePath,
String fileId,
Map<String, List<String>> partitionToReplaceFileIds) throws IOException {
List<HoodieWriteStat> writeStats = new ArrayList<>();
HoodieWriteStat writeStat = createHoodieWriteStat(basePath, commitNumber, partitionPath, filePath, fileId);
writeStats.add(writeStat);
HoodieReplaceCommitMetadata replaceMetadata = new HoodieReplaceCommitMetadata();
replaceMetadata.setPartitionToReplaceFileIds(partitionToReplaceFileIds);
writeStats.forEach(stat -> replaceMetadata.addWriteStat(partitionPath, stat));
File file = basePath.resolve(".hoodie").resolve(commitNumber + ".replacecommit").toFile();
file.createNewFile();
FileOutputStream fileOutputStream = new FileOutputStream(file);
fileOutputStream.write(replaceMetadata.toJsonString().getBytes(StandardCharsets.UTF_8));
fileOutputStream.flush();
fileOutputStream.close();
}
private HoodieWriteStat createHoodieWriteStat(java.nio.file.Path basePath, String commitNumber, String partitionPath, String filePath, String fileId) {
HoodieWriteStat writeStat = new HoodieWriteStat();
writeStat.setFileId(fileId);
writeStat.setNumDeletes(0);
writeStat.setNumUpdateWrites(100);
writeStat.setNumWrites(100);
writeStat.setPath(filePath);
writeStat.setPartitionPath(partitionPath);
writeStat.setTotalLogFilesCompacted(100L);
HoodieWriteStat.RuntimeStats runtimeStats = new HoodieWriteStat.RuntimeStats();
runtimeStats.setTotalScanTime(100);
runtimeStats.setTotalCreateTime(100);
runtimeStats.setTotalUpsertTime(100);
writeStat.setRuntimeStats(runtimeStats);
return writeStat;
}
private void createDeltaCommitFile(
java.nio.file.Path basePath,
String commitNumber,
String partitionPath,
String filePath,
String fileId) throws IOException {
List<HoodieWriteStat> writeStats = new ArrayList<>();
HoodieWriteStat writeStat = createHoodieWriteStat(basePath, commitNumber, partitionPath, filePath, fileId);
writeStats.add(writeStat);
HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata();
writeStats.forEach(stat -> commitMetadata.addWriteStat(partitionPath, stat));
File file = basePath.resolve(".hoodie").resolve(commitNumber + ".deltacommit").toFile();
file.createNewFile();
FileOutputStream fileOutputStream = new FileOutputStream(file);
fileOutputStream.write(commitMetadata.toJsonString().getBytes(StandardCharsets.UTF_8));
fileOutputStream.flush();
fileOutputStream.close();
}
@Test
public void testLogOnlyReader() throws Exception {
// initial commit
Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getEvolvedSchema());
HoodieTestUtils.init(hadoopConf, basePath.toString(), HoodieTableType.MERGE_ON_READ);
String baseInstant = "100";
File partitionDir = InputFormatTestUtil.prepareNonPartitionedParquetTable(basePath, schema, 1, 100, baseInstant,
HoodieTableType.MERGE_ON_READ);
FileCreateUtils.createDeltaCommit(basePath.toString(), baseInstant);
// Add the paths
FileInputFormat.setInputPaths(baseJobConf, partitionDir.getPath());
FileSlice fileSlice = new FileSlice("default", baseInstant, "fileid1");
try {
// update files or generate new log file
int logVersion = 1;
int baseInstantTs = Integer.parseInt(baseInstant);
String instantTime = String.valueOf(baseInstantTs + logVersion);
HoodieLogFormat.Writer writer = InputFormatTestUtil.writeDataBlockToLogFile(partitionDir, fs, schema, "fileid1", baseInstant,
instantTime, 100, 0, logVersion);
long size = writer.getCurrentSize();
writer.close();
assertTrue(size > 0, "block - size should be > 0");
FileCreateUtils.createDeltaCommit(basePath.toString(), instantTime);
// create a split with new log file(s)
fileSlice.addLogFile(writer.getLogFile());
RealtimeFileStatus realtimeFileStatus = new RealtimeFileStatus(new FileStatus(0, false, 1, 1, 0, writer.getLogFile().getPath()));
realtimeFileStatus.setMaxCommitTime(instantTime);
realtimeFileStatus.setBasePath(basePath.toString());
realtimeFileStatus.setDeltaLogPaths(fileSlice.getLogFiles().map(l -> l.getPath().toString()).collect(Collectors.toList()));
PathWithLogFilePath pathWithLogFileStatus = (PathWithLogFilePath) realtimeFileStatus.getPath();
BaseFileWithLogsSplit bs = pathWithLogFileStatus.buildSplit(pathWithLogFileStatus, 0, 0, new String[] {""});
HoodieRealtimeFileSplit split = new HoodieRealtimeFileSplit(bs, bs.getBasePath(), bs.getDeltaLogPaths(), bs.getMaxCommitTime(), Option.empty());
JobConf newJobConf = new JobConf(baseJobConf);
List<Schema.Field> fields = schema.getFields();
setHiveColumnNameProps(fields, newJobConf, false);
// create a dummy RecordReader to be used by HoodieRealtimeRecordReader
RecordReader<NullWritable, ArrayWritable> reader = new HoodieRealtimeRecordReader(split, newJobConf, new HoodieEmptyRecordReader(split, newJobConf));
// use reader to read log file.
NullWritable key = reader.createKey();
ArrayWritable value = reader.createValue();
while (reader.next(key, value)) {
Writable[] values = value.get();
assertEquals(instantTime, values[0].toString());
key = reader.createKey();
value = reader.createValue();
}
reader.close();
} catch (Exception e) {
throw new HoodieException(e.getMessage(), e);
}
}
@Test
public void testIncrementalWithCompaction() throws Exception {
// initial commit
Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getEvolvedSchema());
HoodieTestUtils.init(hadoopConf, basePath.toString(), HoodieTableType.MERGE_ON_READ);
String baseInstant = "100";
File partitionDir = InputFormatTestUtil.prepareParquetTable(basePath, schema, 1, 100, baseInstant,
HoodieTableType.MERGE_ON_READ);
createDeltaCommitFile(basePath, baseInstant,"2016/05/01", "2016/05/01/fileid0_1-0-1_100.parquet", "fileid0");
// Add the paths
FileInputFormat.setInputPaths(baseJobConf, partitionDir.getPath());
createCompactionFile(basePath, "125");
// add inserts after compaction timestamp
InputFormatTestUtil.simulateInserts(partitionDir, ".parquet", "fileId2", 5, "200");
InputFormatTestUtil.commit(basePath, "200");
InputFormatTestUtil.setupIncremental(baseJobConf, "100", 10);
// verify that incremental reads do NOT show inserts after compaction timestamp
HoodieParquetRealtimeInputFormat inputFormat = new HoodieParquetRealtimeInputFormat();
inputFormat.setConf(baseJobConf);
InputSplit[] splits = inputFormat.getSplits(baseJobConf, 1);
assertTrue(splits.length == 0);
}
private File createCompactionFile(java.nio.file.Path basePath, String commitTime)
throws IOException {
File file = basePath.resolve(".hoodie")
.resolve(HoodieTimeline.makeRequestedCompactionFileName(commitTime)).toFile();
assertTrue(file.createNewFile());
FileOutputStream os = new FileOutputStream(file);
try {
HoodieCompactionPlan compactionPlan = HoodieCompactionPlan.newBuilder().setVersion(2).build();
// Write empty commit metadata
os.write(TimelineMetadataUtils.serializeCompactionPlan(compactionPlan).get());
return file;
} finally {
os.close();
}
}
}