[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:
@@ -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)) {
|
||||
|
||||
@@ -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);
|
||||
}
|
||||
@@ -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);
|
||||
|
||||
@@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user