1
0

[HUDI-1828] Update unit tests to support ORC as the base file format (#3237)

This commit is contained in:
Jintao Guan
2021-07-14 09:05:42 -07:00
committed by GitHub
parent 93967404a7
commit 2debb9b3ed
14 changed files with 149 additions and 53 deletions

View File

@@ -23,6 +23,7 @@ import org.apache.hudi.client.common.HoodieSparkEngineContext;
import org.apache.hudi.common.config.HoodieMetadataConfig;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.FileSlice;
import org.apache.hudi.common.model.HoodieFileFormat;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.table.HoodieTableMetaClient;
@@ -30,7 +31,6 @@ import org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner;
import org.apache.hudi.common.table.view.HoodieTableFileSystemView;
import org.apache.hudi.common.table.view.TableFileSystemView;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.ParquetReaderIterator;
import org.apache.hudi.common.util.ValidationUtils;
import org.apache.hudi.config.HoodieMemoryConfig;
@@ -38,8 +38,8 @@ import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord;
import org.apache.hadoop.fs.Path;
import org.apache.parquet.avro.AvroParquetReader;
import org.apache.parquet.avro.AvroReadSupport;
import org.apache.hudi.io.storage.HoodieFileReader;
import org.apache.hudi.io.storage.HoodieFileReaderFactory;
import org.apache.spark.api.java.JavaPairRDD;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
@@ -113,8 +113,14 @@ public class DFSHoodieDatasetInputReader extends DFSDeltaInputReader {
@Override
protected long analyzeSingleFile(String filePath) {
return SparkBasedReader.readParquet(new SparkSession(jsc.sc()), Arrays.asList(filePath),
Option.empty(), Option.empty()).count();
if (filePath.endsWith(HoodieFileFormat.PARQUET.getFileExtension())) {
return SparkBasedReader.readParquet(new SparkSession(jsc.sc()), Arrays.asList(filePath),
Option.empty(), Option.empty()).count();
} else if (filePath.endsWith(HoodieFileFormat.ORC.getFileExtension())) {
return SparkBasedReader.readOrc(new SparkSession(jsc.sc()), Arrays.asList(filePath),
Option.empty(), Option.empty()).count();
}
throw new UnsupportedOperationException("Format for " + filePath + " is not supported yet.");
}
private JavaRDD<GenericRecord> fetchAnyRecordsFromDataset(Option<Long> numRecordsToUpdate) throws IOException {
@@ -149,7 +155,7 @@ public class DFSHoodieDatasetInputReader extends DFSDeltaInputReader {
// TODO : read record count from metadata
// Read the records in a single file
long recordsInSingleFile = iteratorSize(readParquetOrLogFiles(getSingleSliceFromRDD(partitionToFileSlice)));
long recordsInSingleFile = iteratorSize(readColumnarOrLogFiles(getSingleSliceFromRDD(partitionToFileSlice)));
int numFilesToUpdate;
long numRecordsToUpdatePerFile;
if (!numFiles.isPresent() || numFiles.get() <= 0) {
@@ -205,9 +211,9 @@ public class DFSHoodieDatasetInputReader extends DFSDeltaInputReader {
return iteratorLimit(p._2, maxFilesToRead);
}).flatMap(p -> p).repartition(numFiles).map(fileSlice -> {
if (numRecordsToReadPerFile > 0) {
return iteratorLimit(readParquetOrLogFiles(fileSlice), numRecordsToReadPerFile);
return iteratorLimit(readColumnarOrLogFiles(fileSlice), numRecordsToReadPerFile);
} else {
return readParquetOrLogFiles(fileSlice);
return readColumnarOrLogFiles(fileSlice);
}
}).flatMap(p -> p).map(i -> (GenericRecord) i);
}
@@ -253,15 +259,13 @@ public class DFSHoodieDatasetInputReader extends DFSDeltaInputReader {
}).take(1).get(0);
}
private Iterator<IndexedRecord> readParquetOrLogFiles(FileSlice fileSlice) throws IOException {
private Iterator<IndexedRecord> readColumnarOrLogFiles(FileSlice fileSlice) throws IOException {
if (fileSlice.getBaseFile().isPresent()) {
// Read the parquet files using the latest writer schema.
Schema schema = new Schema.Parser().parse(schemaStr);
AvroReadSupport.setAvroReadSchema(metaClient.getHadoopConf(), HoodieAvroUtils.addMetadataFields(schema));
Iterator<IndexedRecord> itr =
new ParquetReaderIterator<IndexedRecord>(AvroParquetReader.<IndexedRecord>builder(new
Path(fileSlice.getBaseFile().get().getPath())).withConf(metaClient.getHadoopConf()).build());
return itr;
// Read the base files using the latest writer schema.
Schema schema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(schemaStr));
HoodieFileReader reader = HoodieFileReaderFactory.getFileReader(metaClient.getHadoopConf(),
new Path(fileSlice.getBaseFile().get().getPath()));
return reader.getRecordIterator(schema);
} else {
// If there is no data file, fall back to reading log files
HoodieMergedLogRecordScanner scanner = HoodieMergedLogRecordScanner.newBuilder()

View File

@@ -67,4 +67,16 @@ public class SparkBasedReader {
.toJavaRDD();
}
public static JavaRDD<GenericRecord> readOrc(SparkSession sparkSession, List<String>
listOfPaths, Option<String> structName, Option<String> nameSpace) {
Dataset<Row> dataSet = sparkSession.read()
.orc((JavaConverters.asScalaIteratorConverter(listOfPaths.iterator()).asScala().toSeq()));
return HoodieSparkUtils
.createRdd(dataSet.toDF(), structName.orElse(RowBasedSchemaProvider.HOODIE_RECORD_STRUCT_NAME),
RowBasedSchemaProvider.HOODIE_RECORD_NAMESPACE)
.toJavaRDD();
}
}