[HUDI-3689] Fix glob path and hive sync in deltastreamer tests (#5117)
* Remove glob pattern basePath from the deltastreamer tests. * [HUDI-3689] Fix file scheme config for CI failure in TestHoodieRealTimeRecordReader Co-authored-by: Raymond Xu <2701446+xushiyan@users.noreply.github.com>
This commit is contained in:
@@ -18,29 +18,6 @@
|
||||
|
||||
package org.apache.hudi.hadoop.realtime;
|
||||
|
||||
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;
|
||||
import org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat;
|
||||
import org.apache.hadoop.hive.serde2.ColumnProjectionUtils;
|
||||
import org.apache.hadoop.io.ArrayWritable;
|
||||
import org.apache.hadoop.io.BooleanWritable;
|
||||
import org.apache.hadoop.io.DoubleWritable;
|
||||
import org.apache.hadoop.io.FloatWritable;
|
||||
import org.apache.hadoop.io.IntWritable;
|
||||
import org.apache.hadoop.io.LongWritable;
|
||||
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;
|
||||
@@ -68,6 +45,30 @@ 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.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;
|
||||
import org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat;
|
||||
import org.apache.hadoop.hive.serde2.ColumnProjectionUtils;
|
||||
import org.apache.hadoop.io.ArrayWritable;
|
||||
import org.apache.hadoop.io.BooleanWritable;
|
||||
import org.apache.hadoop.io.DoubleWritable;
|
||||
import org.apache.hadoop.io.FloatWritable;
|
||||
import org.apache.hadoop.io.IntWritable;
|
||||
import org.apache.hadoop.io.LongWritable;
|
||||
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.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.api.io.TempDir;
|
||||
@@ -78,6 +79,7 @@ import org.junit.jupiter.params.provider.MethodSource;
|
||||
import java.io.File;
|
||||
import java.io.FileOutputStream;
|
||||
import java.io.IOException;
|
||||
import java.net.URI;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
@@ -106,9 +108,11 @@ public class TestHoodieRealtimeRecordReader {
|
||||
@BeforeEach
|
||||
public void setUp() {
|
||||
hadoopConf = HoodieTestUtils.getDefaultHadoopConf();
|
||||
hadoopConf.set("fs.defaultFS", "file:///");
|
||||
hadoopConf.set("fs.file.impl", org.apache.hadoop.fs.LocalFileSystem.class.getName());
|
||||
baseJobConf = new JobConf(hadoopConf);
|
||||
baseJobConf.set(HoodieRealtimeConfig.MAX_DFS_STREAM_BUFFER_SIZE_PROP, String.valueOf(1024 * 1024));
|
||||
fs = FSUtils.getFs(basePath.toString(), baseJobConf);
|
||||
fs = FSUtils.getFs(basePath.toUri().toString(), baseJobConf);
|
||||
}
|
||||
|
||||
@TempDir
|
||||
@@ -810,13 +814,14 @@ public class TestHoodieRealtimeRecordReader {
|
||||
public void testLogOnlyReader() throws Exception {
|
||||
// initial commit
|
||||
Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getEvolvedSchema());
|
||||
HoodieTestUtils.init(hadoopConf, basePath.toString(), HoodieTableType.MERGE_ON_READ);
|
||||
URI baseUri = basePath.toUri();
|
||||
HoodieTestUtils.init(hadoopConf, baseUri.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());
|
||||
FileInputFormat.setInputPaths(baseJobConf, partitionDir.toURI().toString());
|
||||
|
||||
FileSlice fileSlice = new FileSlice("default", baseInstant, "fileid1");
|
||||
try {
|
||||
@@ -836,7 +841,7 @@ public class TestHoodieRealtimeRecordReader {
|
||||
fileSlice.addLogFile(new HoodieLogFile(writer.getLogFile().getPath(), size));
|
||||
RealtimeFileStatus realtimeFileStatus = new RealtimeFileStatus(
|
||||
new FileStatus(writer.getLogFile().getFileSize(), false, 1, 1, 0, writer.getLogFile().getPath()),
|
||||
basePath.toString(),
|
||||
baseUri.toString(),
|
||||
fileSlice.getLogFiles().collect(Collectors.toList()),
|
||||
false,
|
||||
Option.empty());
|
||||
|
||||
Reference in New Issue
Block a user