[HUDI-3280] Cleaning up Hive-related hierarchies after refactoring (#4743)
This commit is contained in:
@@ -18,6 +18,15 @@
|
||||
|
||||
package org.apache.hudi.hadoop;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
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.hadoop.mapreduce.Job;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionPlan;
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.common.model.HoodieCommitMetadata;
|
||||
@@ -34,16 +43,6 @@ import org.apache.hudi.common.testutils.HoodieTestUtils;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.hadoop.testutils.InputFormatTestUtil;
|
||||
import org.apache.hudi.hadoop.utils.HoodieHiveUtils;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
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.hadoop.mapreduce.Job;
|
||||
import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Disabled;
|
||||
@@ -203,11 +202,11 @@ public class TestHoodieParquetInputFormat {
|
||||
FileInputFormat.setInputPaths(jobConf, partitionDir.getPath());
|
||||
InputFormatTestUtil.setupSnapshotIncludePendingCommits(jobConf, "1");
|
||||
Exception exception = assertThrows(HoodieIOException.class, () -> inputFormat.listStatus(jobConf));
|
||||
assertEquals("Valid timestamp is required for hoodie.%s.consume.commit in snapshot mode", exception.getMessage());
|
||||
assertEquals("Query instant (1) not found in the timeline", exception.getMessage());
|
||||
|
||||
InputFormatTestUtil.setupSnapshotMaxCommitTimeQueryMode(jobConf, "1");
|
||||
exception = assertThrows(HoodieIOException.class, () -> inputFormat.listStatus(jobConf));
|
||||
assertEquals("Valid timestamp is required for hoodie.%s.consume.commit in snapshot mode", exception.getMessage());
|
||||
assertEquals("Query instant (1) not found in the timeline", exception.getMessage());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
||||
@@ -18,12 +18,11 @@
|
||||
|
||||
package org.apache.hudi.hadoop.realtime;
|
||||
|
||||
import org.apache.hudi.common.model.HoodieLogFile;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.io.Text;
|
||||
import org.apache.hadoop.mapred.FileSplit;
|
||||
import org.apache.hudi.common.model.HoodieLogFile;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.api.extension.ExtendWith;
|
||||
@@ -72,7 +71,7 @@ public class TestHoodieRealtimeFileSplit {
|
||||
baseFileSplit = new FileSplit(new Path(fileSplitName), 0, 100, new String[] {});
|
||||
maxCommitTime = "10001";
|
||||
|
||||
split = new HoodieRealtimeFileSplit(baseFileSplit, basePath, deltaLogFiles, maxCommitTime, Option.empty());
|
||||
split = new HoodieRealtimeFileSplit(baseFileSplit, basePath, deltaLogFiles, maxCommitTime, false, Option.empty());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
||||
@@ -18,41 +18,10 @@
|
||||
|
||||
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;
|
||||
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.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;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
|
||||
@@ -68,8 +37,36 @@ import org.apache.hadoop.io.NullWritable;
|
||||
import org.apache.hadoop.io.Writable;
|
||||
import org.apache.hadoop.mapred.FileInputFormat;
|
||||
import org.apache.hadoop.mapred.FileSplit;
|
||||
import org.apache.hadoop.mapred.InputSplit;
|
||||
import org.apache.hadoop.mapred.JobConf;
|
||||
import org.apache.hadoop.mapred.RecordReader;
|
||||
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.model.HoodieWriteStat;
|
||||
import org.apache.hudi.common.table.HoodieTableConfig;
|
||||
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;
|
||||
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.RealtimeFileStatus;
|
||||
import org.apache.hudi.hadoop.config.HoodieRealtimeConfig;
|
||||
import org.apache.hudi.hadoop.testutils.InputFormatTestUtil;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.api.io.TempDir;
|
||||
@@ -84,12 +81,12 @@ import java.nio.charset.StandardCharsets;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
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;
|
||||
|
||||
@@ -229,7 +226,9 @@ public class TestHoodieRealtimeRecordReader {
|
||||
new FileSplit(new Path(partitionDir + "/fileid0_1-0-1_" + baseInstant + ".parquet"), 0, 1, baseJobConf),
|
||||
basePath.toUri().toString(), fileSlice.getLogFiles().sorted(HoodieLogFile.getLogFileComparator())
|
||||
.collect(Collectors.toList()),
|
||||
instantTime, Option.empty());
|
||||
instantTime,
|
||||
false,
|
||||
Option.empty());
|
||||
|
||||
// create a RecordReader to be used by HoodieRealtimeRecordReader
|
||||
RecordReader<NullWritable, ArrayWritable> reader = new MapredParquetInputFormat().getRecordReader(
|
||||
@@ -309,7 +308,7 @@ public class TestHoodieRealtimeRecordReader {
|
||||
// create a split with baseFile (parquet file written earlier) and new log file(s)
|
||||
HoodieRealtimeFileSplit split = new HoodieRealtimeFileSplit(
|
||||
new FileSplit(new Path(partitionDir + "/fileid0_1-0-1_" + instantTime + ".parquet"), 0, 1, baseJobConf),
|
||||
basePath.toUri().toString(), Collections.singletonList(writer.getLogFile()), newCommitTime, Option.empty());
|
||||
basePath.toUri().toString(), Collections.singletonList(writer.getLogFile()), newCommitTime, false, Option.empty());
|
||||
|
||||
// create a RecordReader to be used by HoodieRealtimeRecordReader
|
||||
RecordReader<NullWritable, ArrayWritable> reader = new MapredParquetInputFormat().getRecordReader(
|
||||
@@ -388,7 +387,7 @@ public class TestHoodieRealtimeRecordReader {
|
||||
// create a split with baseFile (parquet file written earlier) and new log file(s)
|
||||
HoodieRealtimeFileSplit split = new HoodieRealtimeFileSplit(
|
||||
new FileSplit(new Path(partitionDir + "/fileid0_1-0-1_" + instantTime + ".parquet"), 0, 1, baseJobConf),
|
||||
basePath.toUri().toString(), Collections.singletonList(writer.getLogFile()), newCommitTime, Option.empty());
|
||||
basePath.toUri().toString(), Collections.singletonList(writer.getLogFile()), newCommitTime, false, Option.empty());
|
||||
|
||||
// create a RecordReader to be used by HoodieRealtimeRecordReader
|
||||
RecordReader<NullWritable, ArrayWritable> reader = new MapredParquetInputFormat().getRecordReader(
|
||||
@@ -535,7 +534,7 @@ public class TestHoodieRealtimeRecordReader {
|
||||
// create a split with baseFile (parquet file written earlier) and new log file(s)
|
||||
HoodieRealtimeFileSplit split = new HoodieRealtimeFileSplit(
|
||||
new FileSplit(new Path(partitionDir + "/fileid0_1_" + instantTime + ".parquet"), 0, 1, baseJobConf),
|
||||
basePath.toUri().toString(), logFiles, newCommitTime, Option.empty());
|
||||
basePath.toUri().toString(), logFiles, newCommitTime, false, Option.empty());
|
||||
|
||||
// create a RecordReader to be used by HoodieRealtimeRecordReader
|
||||
RecordReader<NullWritable, ArrayWritable> reader = new MapredParquetInputFormat().getRecordReader(
|
||||
@@ -615,7 +614,7 @@ public class TestHoodieRealtimeRecordReader {
|
||||
HoodieParquetRealtimeInputFormat inputFormat = new HoodieParquetRealtimeInputFormat();
|
||||
inputFormat.setConf(baseJobConf);
|
||||
InputSplit[] splits = inputFormat.getSplits(baseJobConf, 1);
|
||||
assertTrue(splits.length == 1);
|
||||
assertEquals(1, splits.length);
|
||||
JobConf newJobConf = new JobConf(baseJobConf);
|
||||
List<Schema.Field> fields = schema.getFields();
|
||||
setHiveColumnNameProps(fields, newJobConf, false);
|
||||
@@ -769,13 +768,16 @@ public class TestHoodieRealtimeRecordReader {
|
||||
FileCreateUtils.createDeltaCommit(basePath.toString(), instantTime);
|
||||
// create a split with new log file(s)
|
||||
fileSlice.addLogFile(new HoodieLogFile(writer.getLogFile().getPath(), size));
|
||||
RealtimeFileStatus realtimeFileStatus = new RealtimeFileStatus(new FileStatus(writer.getLogFile().getFileSize(), false, 1, 1, 0, writer.getLogFile().getPath()));
|
||||
RealtimeFileStatus realtimeFileStatus = new RealtimeFileStatus(
|
||||
new FileStatus(writer.getLogFile().getFileSize(), false, 1, 1, 0, writer.getLogFile().getPath()),
|
||||
basePath.toString(),
|
||||
fileSlice.getLogFiles().collect(Collectors.toList()),
|
||||
false,
|
||||
Option.empty());
|
||||
realtimeFileStatus.setMaxCommitTime(instantTime);
|
||||
realtimeFileStatus.setBasePath(basePath.toString());
|
||||
realtimeFileStatus.setDeltaLogFiles(fileSlice.getLogFiles().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.getDeltaLogFiles(), bs.getMaxCommitTime(), Option.empty());
|
||||
HoodieRealtimePath realtimePath = (HoodieRealtimePath) realtimeFileStatus.getPath();
|
||||
HoodieRealtimeFileSplit split =
|
||||
new HoodieRealtimeFileSplit(new FileSplit(realtimePath, 0, 0, new String[] {""}), realtimePath);
|
||||
|
||||
JobConf newJobConf = new JobConf(baseJobConf);
|
||||
List<Schema.Field> fields = schema.getFields();
|
||||
|
||||
Reference in New Issue
Block a user