1
0

[HUDI-1055] Remove hardcoded parquet in tests (#2740)

* Remove hardcoded parquet in tests
* Use DataFileUtils.getInstance
* Renaming DataFileUtils to BaseFileUtils

Co-authored-by: Vinoth Chandar <vinoth@apache.org>
This commit is contained in:
TeRS-K
2021-05-11 13:01:45 -04:00
committed by GitHub
parent ac72470e10
commit be9db2c4f5
42 changed files with 359 additions and 218 deletions

View File

@@ -32,6 +32,7 @@ import org.apache.hudi.common.engine.HoodieEngineContext;
import org.apache.hudi.common.model.HoodieFileFormat;
import org.apache.hudi.common.model.HoodieLogFile;
import org.apache.hudi.common.model.HoodiePartitionMetadata;
import org.apache.hudi.common.table.HoodieTableConfig;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.view.FileSystemViewStorageConfig;
@@ -130,7 +131,8 @@ public class FSUtils {
// TODO: this should be removed
public static String makeDataFileName(String instantTime, String writeToken, String fileId) {
return String.format("%s_%s_%s%s", fileId, writeToken, instantTime, HoodieFileFormat.PARQUET.getFileExtension());
return String.format("%s_%s_%s%s", fileId, writeToken, instantTime,
HoodieTableConfig.DEFAULT_BASE_FILE_FORMAT.getFileExtension());
}
public static String makeDataFileName(String instantTime, String writeToken, String fileId, String fileExtension) {
@@ -142,7 +144,7 @@ public class FSUtils {
}
public static String maskWithoutFileId(String instantTime, int taskPartitionId) {
return String.format("*_%s_%s%s", taskPartitionId, instantTime, HoodieFileFormat.PARQUET.getFileExtension());
return String.format("*_%s_%s%s", taskPartitionId, instantTime, HoodieTableConfig.DEFAULT_BASE_FILE_FORMAT.getFileExtension());
}
public static String getCommitFromCommitFile(String commitFileName) {
@@ -329,7 +331,7 @@ public class FSUtils {
}
/**
* Check if the file is a parquet file of a log file. Then get the fileId appropriately.
* Check if the file is a base file of a log file. Then get the fileId appropriately.
*/
public static String getFileIdFromFilePath(Path filePath) {
if (FSUtils.isLogFile(filePath)) {

View File

@@ -0,0 +1,73 @@
/*
* 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 java.util.List;
import java.util.Map;
import java.util.Set;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.common.bloom.BloomFilter;
import org.apache.hudi.common.model.HoodieFileFormat;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.common.table.HoodieTableMetaClient;
public abstract class BaseFileUtils {
public static BaseFileUtils getInstance(String path) {
if (path.endsWith(HoodieFileFormat.PARQUET.getFileExtension())) {
return new ParquetUtils();
}
throw new UnsupportedOperationException("The format for file " + path + " is not supported yet.");
}
public static BaseFileUtils getInstance(HoodieFileFormat fileFormat) {
if (HoodieFileFormat.PARQUET.equals(fileFormat)) {
return new ParquetUtils();
}
throw new UnsupportedOperationException(fileFormat.name() + " format not supported yet.");
}
public static BaseFileUtils getInstance(HoodieTableMetaClient metaClient) {
return getInstance(metaClient.getTableConfig().getBaseFileFormat());
}
public abstract Set<String> readRowKeys(Configuration configuration, Path filePath);
public abstract Set<String> filterRowKeys(Configuration configuration, Path filePath, Set<String> filter);
public abstract List<HoodieKey> fetchRecordKeyPartitionPath(Configuration configuration, Path filePath);
public abstract Schema readAvroSchema(Configuration configuration, Path filePath);
public abstract BloomFilter readBloomFilterFromMetadata(Configuration configuration, Path filePath);
public abstract String[] readMinMaxRecordKeys(Configuration configuration, Path filePath);
public abstract List<GenericRecord> readAvroRecords(Configuration configuration, Path filePath);
public abstract List<GenericRecord> readAvroRecords(Configuration configuration, Path filePath, Schema schema);
public abstract Map<String, String> readFooter(Configuration conf, boolean required, Path orcFilePath,
String... footerNames);
public abstract long getRowCount(Configuration conf, Path filePath);
}

View File

@@ -55,7 +55,7 @@ import java.util.function.Function;
/**
* Utility functions involving with parquet.
*/
public class ParquetUtils {
public class ParquetUtils extends BaseFileUtils {
/**
* Read the rowKey list from the given parquet file.
@@ -64,8 +64,9 @@ public class ParquetUtils {
* @param configuration configuration to build fs object
* @return Set Set of row keys
*/
public static Set<String> readRowKeysFromParquet(Configuration configuration, Path filePath) {
return filterParquetRowKeys(configuration, filePath, new HashSet<>());
@Override
public Set<String> readRowKeys(Configuration configuration, Path filePath) {
return filterRowKeys(configuration, filePath, new HashSet<>());
}
/**
@@ -77,7 +78,8 @@ public class ParquetUtils {
* @param filter record keys filter
* @return Set Set of row keys matching candidateRecordKeys
*/
public static Set<String> filterParquetRowKeys(Configuration configuration, Path filePath, Set<String> filter) {
@Override
public Set<String> filterRowKeys(Configuration configuration, Path filePath, Set<String> filter) {
return filterParquetRowKeys(configuration, filePath, filter, HoodieAvroUtils.getRecordKeySchema());
}
@@ -128,7 +130,8 @@ public class ParquetUtils {
* @param configuration configuration to build fs object
* @return {@link List} of {@link HoodieKey}s fetched from the parquet file
*/
public static List<HoodieKey> fetchRecordKeyPartitionPathFromParquet(Configuration configuration, Path filePath) {
@Override
public List<HoodieKey> fetchRecordKeyPartitionPath(Configuration configuration, Path filePath) {
List<HoodieKey> hoodieKeys = new ArrayList<>();
try {
if (!filePath.getFileSystem(configuration).exists(filePath)) {
@@ -156,7 +159,7 @@ public class ParquetUtils {
return hoodieKeys;
}
public static ParquetMetadata readMetadata(Configuration conf, Path parquetFilePath) {
public ParquetMetadata readMetadata(Configuration conf, Path parquetFilePath) {
ParquetMetadata footer;
try {
// TODO(vc): Should we use the parallel reading version here?
@@ -170,11 +173,12 @@ public class ParquetUtils {
/**
* Get the schema of the given parquet file.
*/
public static MessageType readSchema(Configuration configuration, Path parquetFilePath) {
public MessageType readSchema(Configuration configuration, Path parquetFilePath) {
return readMetadata(configuration, parquetFilePath).getFileMetaData().getSchema();
}
private static Map<String, String> readParquetFooter(Configuration configuration, boolean required,
@Override
public Map<String, String> readFooter(Configuration configuration, boolean required,
Path parquetFilePath, String... footerNames) {
Map<String, String> footerVals = new HashMap<>();
ParquetMetadata footer = readMetadata(configuration, parquetFilePath);
@@ -190,16 +194,18 @@ public class ParquetUtils {
return footerVals;
}
public static Schema readAvroSchema(Configuration configuration, Path parquetFilePath) {
@Override
public Schema readAvroSchema(Configuration configuration, Path parquetFilePath) {
return new AvroSchemaConverter(configuration).convert(readSchema(configuration, parquetFilePath));
}
/**
* Read out the bloom filter from the parquet file meta data.
*/
public static BloomFilter readBloomFilterFromParquetMetadata(Configuration configuration, Path parquetFilePath) {
@Override
public BloomFilter readBloomFilterFromMetadata(Configuration configuration, Path parquetFilePath) {
Map<String, String> footerVals =
readParquetFooter(configuration, false, parquetFilePath,
readFooter(configuration, false, parquetFilePath,
HoodieAvroWriteSupport.HOODIE_AVRO_BLOOM_FILTER_METADATA_KEY,
HoodieAvroWriteSupport.OLD_HOODIE_AVRO_BLOOM_FILTER_METADATA_KEY,
HoodieAvroWriteSupport.HOODIE_BLOOM_FILTER_TYPE_CODE);
@@ -220,8 +226,9 @@ public class ParquetUtils {
return toReturn;
}
public static String[] readMinMaxRecordKeys(Configuration configuration, Path parquetFilePath) {
Map<String, String> minMaxKeys = readParquetFooter(configuration, true, parquetFilePath,
@Override
public String[] readMinMaxRecordKeys(Configuration configuration, Path parquetFilePath) {
Map<String, String> minMaxKeys = readFooter(configuration, true, parquetFilePath,
HoodieAvroWriteSupport.HOODIE_MIN_RECORD_KEY_FOOTER, HoodieAvroWriteSupport.HOODIE_MAX_RECORD_KEY_FOOTER);
if (minMaxKeys.size() != 2) {
throw new HoodieException(
@@ -235,7 +242,8 @@ public class ParquetUtils {
/**
* NOTE: This literally reads the entire file contents, thus should be used with caution.
*/
public static List<GenericRecord> readAvroRecords(Configuration configuration, Path filePath) {
@Override
public List<GenericRecord> readAvroRecords(Configuration configuration, Path filePath) {
ParquetReader reader = null;
List<GenericRecord> records = new ArrayList<>();
try {
@@ -262,13 +270,20 @@ public class ParquetUtils {
return records;
}
@Override
public List<GenericRecord> readAvroRecords(Configuration configuration, Path filePath, Schema schema) {
AvroReadSupport.setAvroReadSchema(configuration, schema);
return readAvroRecords(configuration, filePath);
}
/**
* Returns the number of records in the parquet file.
*
* @param conf Configuration
* @param parquetFilePath path of the file
*/
public static long getRowCount(Configuration conf, Path parquetFilePath) {
@Override
public long getRowCount(Configuration conf, Path parquetFilePath) {
ParquetMetadata footer;
long rowCount = 0;
footer = readMetadata(conf, parquetFilePath);

View File

@@ -27,8 +27,9 @@ import org.apache.avro.generic.IndexedRecord;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.common.bloom.BloomFilter;
import org.apache.hudi.common.model.HoodieFileFormat;
import org.apache.hudi.common.util.BaseFileUtils;
import org.apache.hudi.common.util.ParquetReaderIterator;
import org.apache.hudi.common.util.ParquetUtils;
import org.apache.parquet.avro.AvroParquetReader;
import org.apache.parquet.avro.AvroReadSupport;
import org.apache.parquet.hadoop.ParquetReader;
@@ -36,24 +37,26 @@ import org.apache.parquet.hadoop.ParquetReader;
public class HoodieParquetReader<R extends IndexedRecord> implements HoodieFileReader {
private Path path;
private Configuration conf;
private final BaseFileUtils parquetUtils;
public HoodieParquetReader(Configuration configuration, Path path) {
this.conf = configuration;
this.path = path;
this.parquetUtils = BaseFileUtils.getInstance(HoodieFileFormat.PARQUET);
}
public String[] readMinMaxRecordKeys() {
return ParquetUtils.readMinMaxRecordKeys(conf, path);
return parquetUtils.readMinMaxRecordKeys(conf, path);
}
@Override
public BloomFilter readBloomFilter() {
return ParquetUtils.readBloomFilterFromParquetMetadata(conf, path);
return parquetUtils.readBloomFilterFromMetadata(conf, path);
}
@Override
public Set<String> filterRowKeys(Set candidateRowKeys) {
return ParquetUtils.filterParquetRowKeys(conf, path, candidateRowKeys);
return parquetUtils.filterRowKeys(conf, path, candidateRowKeys);
}
@Override
@@ -65,7 +68,7 @@ public class HoodieParquetReader<R extends IndexedRecord> implements HoodieFileR
@Override
public Schema getSchema() {
return ParquetUtils.readAvroSchema(conf, path);
return parquetUtils.readAvroSchema(conf, path);
}
@Override
@@ -74,6 +77,6 @@ public class HoodieParquetReader<R extends IndexedRecord> implements HoodieFileR
@Override
public long getTotalRecords() {
return ParquetUtils.getRowCount(conf, path);
return parquetUtils.getRowCount(conf, path);
}
}

View File

@@ -26,6 +26,7 @@ import org.apache.hudi.common.bootstrap.index.BootstrapIndex.IndexWriter;
import org.apache.hudi.common.bootstrap.index.HFileBootstrapIndex;
import org.apache.hudi.common.model.BootstrapFileMapping;
import org.apache.hudi.common.model.HoodieFileGroupId;
import org.apache.hudi.common.table.HoodieTableConfig;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.testutils.HoodieCommonTestHarness;
import org.apache.hudi.common.util.collection.Pair;
@@ -62,7 +63,7 @@ public class TestBootstrapIndex extends HoodieCommonTestHarness {
private static final String[] PARTITIONS = {"2020/03/18", "2020/03/19", "2020/03/20", "2020/03/21"};
private static final Set<String> PARTITION_SET = Arrays.stream(PARTITIONS).collect(Collectors.toSet());
private static final String BOOTSTRAP_BASE_PATH = "/tmp/source/parquet_tables/table1";
private static final String BOOTSTRAP_BASE_PATH = "/tmp/source/data_tables/table1";
@BeforeEach
public void init() throws IOException {
@@ -168,7 +169,7 @@ public class TestBootstrapIndex extends HoodieCommonTestHarness {
return Arrays.stream(partitions).map(partition -> {
return Pair.of(partition, IntStream.range(0, numEntriesPerPartition).mapToObj(idx -> {
String hudiFileId = UUID.randomUUID().toString();
String sourceFileName = idx + ".parquet";
String sourceFileName = idx + HoodieTableConfig.DEFAULT_BASE_FILE_FORMAT.getFileExtension();
HoodieFileStatus sourceFileStatus = HoodieFileStatus.newBuilder()
.setPath(HoodiePath.newBuilder().setUri(sourceBasePath + "/" + partition + "/" + sourceFileName).build())
.setLength(256 * 1024 * 1024L)

View File

@@ -21,6 +21,7 @@ package org.apache.hudi.common.fs;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.common.model.HoodieLogFile;
import org.apache.hudi.common.table.HoodieTableConfig;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.testutils.HoodieCommonTestHarness;
import org.apache.hudi.common.testutils.HoodieTestUtils;
@@ -56,6 +57,7 @@ public class TestFSUtils extends HoodieCommonTestHarness {
private final long minCleanToKeep = 10;
private static String TEST_WRITE_TOKEN = "1-0-1";
private static final String BASE_FILE_EXTENSION = HoodieTableConfig.DEFAULT_BASE_FILE_FORMAT.getFileExtension();
@Rule
public final EnvironmentVariables environmentVariables = new EnvironmentVariables();
@@ -69,14 +71,14 @@ public class TestFSUtils extends HoodieCommonTestHarness {
public void testMakeDataFileName() {
String instantTime = COMMIT_FORMATTER.format(new Date());
String fileName = UUID.randomUUID().toString();
assertEquals(FSUtils.makeDataFileName(instantTime, TEST_WRITE_TOKEN, fileName), fileName + "_" + TEST_WRITE_TOKEN + "_" + instantTime + ".parquet");
assertEquals(FSUtils.makeDataFileName(instantTime, TEST_WRITE_TOKEN, fileName), fileName + "_" + TEST_WRITE_TOKEN + "_" + instantTime + BASE_FILE_EXTENSION);
}
@Test
public void testMaskFileName() {
String instantTime = COMMIT_FORMATTER.format(new Date());
int taskPartitionId = 2;
assertEquals(FSUtils.maskWithoutFileId(instantTime, taskPartitionId), "*_" + taskPartitionId + "_" + instantTime + ".parquet");
assertEquals(FSUtils.maskWithoutFileId(instantTime, taskPartitionId), "*_" + taskPartitionId + "_" + instantTime + BASE_FILE_EXTENSION);
}
@Test
@@ -100,9 +102,12 @@ public class TestFSUtils extends HoodieCommonTestHarness {
});
// Files inside partitions and marker directories
List<String> files = Arrays.asList("2016/04/15/1_1-0-1_20190528120000.parquet",
"2016/05/16/2_1-0-1_20190528120000.parquet", ".hoodie/.temp/2/2016/05/16/2_1-0-1_20190528120000.parquet",
".hoodie/.temp/2/2016/04/15/1_1-0-1_20190528120000.parquet");
List<String> files = Stream.of("2016/04/15/1_1-0-1_20190528120000",
"2016/05/16/2_1-0-1_20190528120000",
".hoodie/.temp/2/2016/05/16/2_1-0-1_20190528120000",
".hoodie/.temp/2/2016/04/15/1_1-0-1_20190528120000")
.map(fileName -> fileName + BASE_FILE_EXTENSION)
.collect(Collectors.toList());
files.forEach(f -> {
try {

View File

@@ -18,6 +18,7 @@
package org.apache.hudi.common.model;
import org.apache.hudi.common.table.HoodieTableConfig;
import org.junit.jupiter.api.Test;
import java.util.ArrayList;
@@ -33,7 +34,7 @@ public class TestHoodieDeltaWriteStat {
@Test
public void testBaseFileAndLogFiles() {
HoodieDeltaWriteStat writeStat = new HoodieDeltaWriteStat();
String baseFile = "file1.parquet";
String baseFile = "file1" + HoodieTableConfig.DEFAULT_BASE_FILE_FORMAT.getFileExtension();
String logFile1 = ".log1.log";
String logFile2 = ".log2.log";

View File

@@ -284,7 +284,7 @@ public class TestTimelineUtils extends HoodieCommonTestHarness {
HoodieWriteStat stat = new HoodieWriteStat();
stat.setFileId(i + "");
stat.setPartitionPath(Paths.get(basePath, partition).toString());
stat.setPath(commitTs + "." + i + ".parquet");
stat.setPath(commitTs + "." + i + metaClient.getTableConfig().getBaseFileFormat().getFileExtension());
commit.addWriteStat(partition, stat);
}
for (Map.Entry<String, String> extraEntries : extraMetadata.entrySet()) {
@@ -303,7 +303,7 @@ public class TestTimelineUtils extends HoodieCommonTestHarness {
HoodieWriteStat stat = new HoodieWriteStat();
stat.setFileId(i + "");
stat.setPartitionPath(Paths.get(basePath, newFilePartition).toString());
stat.setPath(commitTs + "." + i + ".parquet");
stat.setPath(commitTs + "." + i + metaClient.getTableConfig().getBaseFileFormat().getFileExtension());
commit.addWriteStat(newFilePartition, stat);
}
Map<String, List<String>> partitionToReplaceFileIds = new HashMap<>();

View File

@@ -306,7 +306,7 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
String partitionPath = "2016/05/01";
new File(basePath + "/" + partitionPath).mkdirs();
String fileId = UUID.randomUUID().toString();
String srcName = "part_0000.parquet";
String srcName = "part_0000" + metaClient.getTableConfig().getBaseFileFormat().getFileExtension();
HoodieFileStatus srcFileStatus = HoodieFileStatus.newBuilder()
.setPath(HoodiePath.newBuilder().setUri(BOOTSTRAP_SOURCE_PATH + partitionPath + "/" + srcName).build())
.setLength(256 * 1024 * 1024L)

View File

@@ -22,6 +22,7 @@ import org.apache.hudi.common.model.CompactionOperation;
import org.apache.hudi.common.model.FileSlice;
import org.apache.hudi.common.model.HoodieBaseFile;
import org.apache.hudi.common.model.HoodieFileGroup;
import org.apache.hudi.common.table.HoodieTableConfig;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.common.testutils.MockHoodieTimeline;
@@ -66,7 +67,8 @@ public class TestPriorityBasedFileSystemView {
public void setUp() {
fsView = new PriorityBasedFileSystemView(primary, secondary);
testBaseFileStream = Stream.of(new HoodieBaseFile("test"));
testFileSliceStream = Stream.of(new FileSlice("2020-01-01", "20:20", "file0001.parquet"));
testFileSliceStream = Stream.of(new FileSlice("2020-01-01", "20:20",
"file0001" + HoodieTableConfig.DEFAULT_BASE_FILE_FORMAT.getFileExtension()));
}
private void resetMocks() {

View File

@@ -29,6 +29,7 @@ import org.apache.hudi.common.model.HoodieFileFormat;
import org.apache.hudi.common.model.HoodiePartitionMetadata;
import org.apache.hudi.common.model.HoodieReplaceCommitMetadata;
import org.apache.hudi.common.model.IOType;
import org.apache.hudi.common.table.HoodieTableConfig;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.common.table.view.HoodieTableFileSystemView;
@@ -54,9 +55,10 @@ import static org.apache.hudi.common.table.timeline.TimelineMetadataUtils.serial
public class FileCreateUtils {
private static final String WRITE_TOKEN = "1-0-1";
private static final String BASE_FILE_EXTENSION = HoodieTableConfig.DEFAULT_BASE_FILE_FORMAT.getFileExtension();
public static String baseFileName(String instantTime, String fileId) {
return baseFileName(instantTime, fileId, HoodieFileFormat.PARQUET.getFileExtension());
return baseFileName(instantTime, fileId, BASE_FILE_EXTENSION);
}
public static String baseFileName(String instantTime, String fileId, String fileExtension) {
@@ -72,7 +74,7 @@ public class FileCreateUtils {
}
public static String markerFileName(String instantTime, String fileId, IOType ioType) {
return markerFileName(instantTime, fileId, ioType, HoodieFileFormat.PARQUET.getFileExtension());
return markerFileName(instantTime, fileId, ioType, BASE_FILE_EXTENSION);
}
public static String markerFileName(String instantTime, String fileId, IOType ioType, String fileExtension) {

View File

@@ -29,6 +29,7 @@ import org.apache.hudi.common.model.HoodieCommitMetadata;
import org.apache.hudi.common.model.HoodieFileFormat;
import org.apache.hudi.common.model.HoodieReplaceCommitMetadata;
import org.apache.hudi.common.model.IOType;
import org.apache.hudi.common.table.HoodieTableConfig;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
@@ -437,7 +438,7 @@ public class HoodieTestTable {
}
public FileStatus[] listAllBaseFiles() throws IOException {
return listAllBaseFiles(HoodieFileFormat.PARQUET.getFileExtension());
return listAllBaseFiles(HoodieTableConfig.DEFAULT_BASE_FILE_FORMAT.getFileExtension());
}
public FileStatus[] listAllBaseFiles(String fileExtension) throws IOException {

View File

@@ -97,6 +97,8 @@ public class TestCompactionUtils extends HoodieCommonTestHarness {
@Test
public void testBuildFromFileSlice() {
String extension = metaClient.getTableConfig().getBaseFileFormat().getFileExtension();
// Empty File-Slice with no data and log files
FileSlice emptyFileSlice = new FileSlice(DEFAULT_PARTITION_PATHS[0], "000", "empty1");
HoodieCompactionOperation op =
@@ -106,7 +108,7 @@ public class TestCompactionUtils extends HoodieCommonTestHarness {
// File Slice with data-file but no log files
FileSlice noLogFileSlice = new FileSlice(DEFAULT_PARTITION_PATHS[0], "000", "noLog1");
noLogFileSlice.setBaseFile(new DummyHoodieBaseFile("/tmp/noLog_1_000.parquet"));
noLogFileSlice.setBaseFile(new DummyHoodieBaseFile("/tmp/noLog_1_000" + extension));
op = CompactionUtils.buildFromFileSlice(DEFAULT_PARTITION_PATHS[0], noLogFileSlice, Option.of(metricsCaptureFn));
testFileSliceCompactionOpEquality(noLogFileSlice, op, DEFAULT_PARTITION_PATHS[0],
LATEST_COMPACTION_METADATA_VERSION);
@@ -122,7 +124,7 @@ public class TestCompactionUtils extends HoodieCommonTestHarness {
// File Slice with data-file and log files present
FileSlice fileSlice = new FileSlice(DEFAULT_PARTITION_PATHS[0], "000", "noData1");
fileSlice.setBaseFile(new DummyHoodieBaseFile("/tmp/noLog_1_000.parquet"));
fileSlice.setBaseFile(new DummyHoodieBaseFile("/tmp/noLog_1_000" + extension));
fileSlice.addLogFile(
new HoodieLogFile(new Path(FSUtils.makeLogFileName("noData1", ".log", "000", 1, TEST_WRITE_TOKEN))));
fileSlice.addLogFile(
@@ -135,16 +137,18 @@ public class TestCompactionUtils extends HoodieCommonTestHarness {
* Generate input for compaction plan tests.
*/
private Pair<List<Pair<String, FileSlice>>, HoodieCompactionPlan> buildCompactionPlan() {
String extension = metaClient.getTableConfig().getBaseFileFormat().getFileExtension();
Path fullPartitionPath = new Path(new Path(metaClient.getBasePath()), DEFAULT_PARTITION_PATHS[0]);
FileSlice emptyFileSlice = new FileSlice(DEFAULT_PARTITION_PATHS[0], "000", "empty1");
FileSlice fileSlice = new FileSlice(DEFAULT_PARTITION_PATHS[0], "000", "noData1");
fileSlice.setBaseFile(new DummyHoodieBaseFile(fullPartitionPath.toString() + "/data1_1_000.parquet"));
fileSlice.setBaseFile(new DummyHoodieBaseFile(fullPartitionPath.toString() + "/data1_1_000" + extension));
fileSlice.addLogFile(new HoodieLogFile(
new Path(fullPartitionPath, new Path(FSUtils.makeLogFileName("noData1", ".log", "000", 1, TEST_WRITE_TOKEN)))));
fileSlice.addLogFile(new HoodieLogFile(
new Path(fullPartitionPath, new Path(FSUtils.makeLogFileName("noData1", ".log", "000", 2, TEST_WRITE_TOKEN)))));
FileSlice noLogFileSlice = new FileSlice(DEFAULT_PARTITION_PATHS[0], "000", "noLog1");
noLogFileSlice.setBaseFile(new DummyHoodieBaseFile(fullPartitionPath.toString() + "/noLog_1_000.parquet"));
noLogFileSlice.setBaseFile(new DummyHoodieBaseFile(fullPartitionPath.toString() + "/noLog_1_000" + extension));
FileSlice noDataFileSlice = new FileSlice(DEFAULT_PARTITION_PATHS[0], "000", "noData1");
noDataFileSlice.addLogFile(new HoodieLogFile(
new Path(fullPartitionPath, new Path(FSUtils.makeLogFileName("noData1", ".log", "000", 1, TEST_WRITE_TOKEN)))));

View File

@@ -58,6 +58,8 @@ import static org.junit.jupiter.api.Assertions.assertTrue;
*/
public class TestParquetUtils extends HoodieCommonTestHarness {
private ParquetUtils parquetUtils = new ParquetUtils();
public static List<Arguments> bloomFilterTypeCodes() {
return Arrays.asList(
Arguments.of(BloomFilterTypeCode.SIMPLE.name()),
@@ -83,13 +85,13 @@ public class TestParquetUtils extends HoodieCommonTestHarness {
// Read and verify
List<String> rowKeysInFile = new ArrayList<>(
ParquetUtils.readRowKeysFromParquet(HoodieTestUtils.getDefaultHadoopConf(), new Path(filePath)));
parquetUtils.readRowKeys(HoodieTestUtils.getDefaultHadoopConf(), new Path(filePath)));
Collections.sort(rowKeysInFile);
Collections.sort(rowKeys);
assertEquals(rowKeys, rowKeysInFile, "Did not read back the expected list of keys");
BloomFilter filterInFile =
ParquetUtils.readBloomFilterFromParquetMetadata(HoodieTestUtils.getDefaultHadoopConf(), new Path(filePath));
parquetUtils.readBloomFilterFromMetadata(HoodieTestUtils.getDefaultHadoopConf(), new Path(filePath));
for (String rowKey : rowKeys) {
assertTrue(filterInFile.mightContain(rowKey), "key should be found in bloom filter");
}
@@ -113,7 +115,7 @@ public class TestParquetUtils extends HoodieCommonTestHarness {
// Read and verify
Set<String> filtered =
ParquetUtils.filterParquetRowKeys(HoodieTestUtils.getDefaultHadoopConf(), new Path(filePath), filter);
parquetUtils.filterRowKeys(HoodieTestUtils.getDefaultHadoopConf(), new Path(filePath), filter);
assertEquals(filter.size(), filtered.size(), "Filtered count does not match");
@@ -140,7 +142,7 @@ public class TestParquetUtils extends HoodieCommonTestHarness {
// Read and verify
List<HoodieKey> fetchedRows =
ParquetUtils.fetchRecordKeyPartitionPathFromParquet(HoodieTestUtils.getDefaultHadoopConf(), new Path(filePath));
parquetUtils.fetchRecordKeyPartitionPath(HoodieTestUtils.getDefaultHadoopConf(), new Path(filePath));
assertEquals(rowKeys.size(), fetchedRows.size(), "Total count does not match");
for (HoodieKey entry : fetchedRows) {
@@ -157,7 +159,7 @@ public class TestParquetUtils extends HoodieCommonTestHarness {
}
writeParquetFile(BloomFilterTypeCode.SIMPLE.name(), filePath, rowKeys);
assertEquals(123, ParquetUtils.getRowCount(HoodieTestUtils.getDefaultHadoopConf(), new Path(filePath)));
assertEquals(123, parquetUtils.getRowCount(HoodieTestUtils.getDefaultHadoopConf(), new Path(filePath)));
}
private void writeParquetFile(String typeCode, String filePath, List<String> rowKeys) throws Exception {

View File

@@ -21,6 +21,7 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.common.model.HoodiePartitionMetadata;
import org.apache.hudi.common.table.HoodieTableConfig;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.junit.jupiter.api.BeforeAll;
import org.junit.jupiter.api.Test;
@@ -36,6 +37,7 @@ import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertTrue;
public final class TestTablePathUtils {
private static final String BASE_FILE_EXTENSION = HoodieTableConfig.DEFAULT_BASE_FILE_FORMAT.getFileExtension();
@TempDir
static File tempDir;
@@ -73,9 +75,9 @@ public final class TestTablePathUtils {
partitionMetadata2.trySave(2);
// Create files
URI filePathURI1 = Paths.get(partitionPathURI1.getPath(), "data1.parquet").toUri();
URI filePathURI1 = Paths.get(partitionPathURI1.getPath(), "data1" + BASE_FILE_EXTENSION).toUri();
filePath1 = new Path(filePathURI1);
URI filePathURI2 = Paths.get(partitionPathURI2.getPath(), "data2.parquet").toUri();
URI filePathURI2 = Paths.get(partitionPathURI2.getPath(), "data2" + BASE_FILE_EXTENSION).toUri();
filePath2 = new Path(filePathURI2);
assertTrue(new File(filePathURI1).createNewFile());