1
0

[HUDI-684] Introduced abstraction for writing and reading different types of base file formats. (#1687)

Notable changes:
    1. HoodieFileWriter and HoodieFileReader abstractions for writer/reader side of a base file format
    2. HoodieDataBlock abstraction for creation specific data blocks for base file formats. (e.g. Parquet has HoodieAvroDataBlock)
    3. All hardocded references to Parquet / Parquet based classes have been abstracted to call methods which accept a base file format
    4. HiveSyncTool accepts the base file format as a CLI parameter
    5. HoodieDeltaStreamer accepts the base file format as a CLI parameter
    6. HoodieSparkSqlWriter accepts the base file format as a parameter
This commit is contained in:
Prashant Wason
2020-06-25 23:46:55 -07:00
committed by GitHub
parent 5e47673341
commit 2603cfb33e
55 changed files with 1086 additions and 466 deletions

View File

@@ -107,15 +107,21 @@ public class FSUtils {
return String.format("%d-%d-%d", taskPartitionId, stageId, taskAttemptId);
}
// TODO: this should be removed
public static String makeDataFileName(String instantTime, String writeToken, String fileId) {
return String.format("%s_%s_%s.parquet", fileId, writeToken, instantTime);
return String.format("%s_%s_%s%s", fileId, writeToken, instantTime, HoodieFileFormat.PARQUET.getFileExtension());
}
public static String makeDataFileName(String instantTime, String writeToken, String fileId, String fileExtension) {
return String.format("%s_%s_%s%s", fileId, writeToken, instantTime, fileExtension);
}
public static String makeMarkerFile(String instantTime, String writeToken, String fileId) {
return String.format("%s_%s_%s%s", fileId, writeToken, instantTime, HoodieTableMetaClient.MARKER_EXTN);
}
public static String translateMarkerToDataPath(String basePath, String markerPath, String instantTs) {
public static String translateMarkerToDataPath(String basePath, String markerPath, String instantTs,
String baseFileExtension) {
ValidationUtils.checkArgument(markerPath.endsWith(HoodieTableMetaClient.MARKER_EXTN));
String markerRootPath = Path.getPathWithoutSchemeAndAuthority(
new Path(String.format("%s/%s/%s", basePath, HoodieTableMetaClient.TEMPFOLDER_NAME, instantTs))).toString();
@@ -123,8 +129,7 @@ public class FSUtils {
ValidationUtils.checkArgument(begin >= 0,
"Not in marker dir. Marker Path=" + markerPath + ", Expected Marker Root=" + markerRootPath);
String rPath = markerPath.substring(begin + markerRootPath.length() + 1);
return String.format("%s/%s%s", basePath, rPath.replace(HoodieTableMetaClient.MARKER_EXTN, ""),
HoodieFileFormat.PARQUET.getFileExtension());
return String.format("%s/%s%s", basePath, rPath.replace(HoodieTableMetaClient.MARKER_EXTN, ""), baseFileExtension);
}
public static String maskWithoutFileId(String instantTime, int taskPartitionId) {
@@ -195,12 +200,12 @@ public class FSUtils {
}
public static List<String> getAllDataFilesForMarkers(FileSystem fs, String basePath, String instantTs,
String markerDir) throws IOException {
String markerDir, String baseFileExtension) throws IOException {
List<String> dataFiles = new LinkedList<>();
processFiles(fs, markerDir, (status) -> {
String pathStr = status.getPath().toString();
if (pathStr.endsWith(HoodieTableMetaClient.MARKER_EXTN)) {
dataFiles.add(FSUtils.translateMarkerToDataPath(basePath, pathStr, instantTs));
dataFiles.add(FSUtils.translateMarkerToDataPath(basePath, pathStr, instantTs, baseFileExtension));
}
return true;
}, false);
@@ -545,4 +550,13 @@ public class FSUtils {
|| inputStream.getWrappedStream().getClass().getCanonicalName()
.equals("com.google.cloud.hadoop.fs.gcs.GoogleHadoopFSInputStream");
}
public static Configuration registerFileSystem(Path file, Configuration conf) {
Configuration returnConf = new Configuration(conf);
String scheme = FSUtils.getFs(file.toString(), conf).getScheme();
returnConf.set("fs." + HoodieWrapperFileSystem.getHoodieScheme(scheme) + ".impl",
HoodieWrapperFileSystem.class.getName());
return returnConf;
}
}

View File

@@ -128,7 +128,8 @@ public class HoodieTableMetaClient implements Serializable {
}
this.timelineLayoutVersion = layoutVersion.orElseGet(() -> tableConfig.getTimelineLayoutVersion().get());
this.loadActiveTimelineOnLoad = loadActiveTimelineOnLoad;
LOG.info("Finished Loading Table of type " + tableType + "(version=" + timelineLayoutVersion + ") from " + basePath);
LOG.info("Finished Loading Table of type " + tableType + "(version=" + timelineLayoutVersion + ", baseFileFormat="
+ this.tableConfig.getBaseFileFormat() + ") from " + basePath);
if (loadActiveTimelineOnLoad) {
LOG.info("Loading Active commit timeline for " + basePath);
getActiveTimeline();
@@ -299,12 +300,22 @@ public class HoodieTableMetaClient implements Serializable {
}
/**
* Helper method to initialize a table, with given basePath, tableType, name, archiveFolder.
* Helper method to initialize a table, with given basePath, tableType, name, archiveFolder, payloadClass.
*/
public static HoodieTableMetaClient initTableType(Configuration hadoopConf, String basePath, String tableType,
String tableName, String archiveLogFolder, String payloadClassName) throws IOException {
return initTableType(hadoopConf, basePath, HoodieTableType.valueOf(tableType), tableName,
archiveLogFolder, payloadClassName, null);
archiveLogFolder, payloadClassName, null, null);
}
/**
* Helper method to initialize a table, with given basePath, tableType, name, archiveFolder, payloadClass and
* base file format.
*/
public static HoodieTableMetaClient initTableType(Configuration hadoopConf, String basePath, String tableType,
String tableName, String archiveLogFolder, String payloadClassName, String baseFileFormat) throws IOException {
return initTableType(hadoopConf, basePath, HoodieTableType.valueOf(tableType), tableName,
archiveLogFolder, payloadClassName, null, baseFileFormat);
}
/**
@@ -312,12 +323,20 @@ public class HoodieTableMetaClient implements Serializable {
*/
public static HoodieTableMetaClient initTableType(Configuration hadoopConf, String basePath,
HoodieTableType tableType, String tableName, String payloadClassName) throws IOException {
return initTableType(hadoopConf, basePath, tableType, tableName, null, payloadClassName, null);
return initTableType(hadoopConf, basePath, tableType, tableName, null, payloadClassName, null, null);
}
public static HoodieTableMetaClient initTableType(Configuration hadoopConf, String basePath,
HoodieTableType tableType, String tableName,
String archiveLogFolder, String payloadClassName,
Integer timelineLayoutVersion) throws IOException {
return initTableType(hadoopConf, basePath, tableType, tableName, archiveLogFolder, payloadClassName,
timelineLayoutVersion, null);
}
public static HoodieTableMetaClient initTableType(Configuration hadoopConf, String basePath,
HoodieTableType tableType, String tableName, String archiveLogFolder, String payloadClassName,
Integer timelineLayoutVersion) throws IOException {
Integer timelineLayoutVersion, String baseFileFormat) throws IOException {
Properties properties = new Properties();
properties.setProperty(HoodieTableConfig.HOODIE_TABLE_NAME_PROP_NAME, tableName);
properties.setProperty(HoodieTableConfig.HOODIE_TABLE_TYPE_PROP_NAME, tableType.name());
@@ -332,6 +351,10 @@ public class HoodieTableMetaClient implements Serializable {
if (null != timelineLayoutVersion) {
properties.put(HoodieTableConfig.HOODIE_TIMELINE_LAYOUT_VERSION, String.valueOf(timelineLayoutVersion));
}
if (null != baseFileFormat) {
properties.setProperty(HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP_NAME, baseFileFormat.toUpperCase());
}
return HoodieTableMetaClient.initTableAndGetMetaClient(hadoopConf, basePath, properties);
}

View File

@@ -31,7 +31,7 @@ import org.apache.hudi.common.model.HoodieFileFormat;
import org.apache.hudi.common.model.HoodieLogFile;
import org.apache.hudi.common.table.log.HoodieLogFormat;
import org.apache.hudi.common.table.log.HoodieLogFormat.Reader;
import org.apache.hudi.common.table.log.block.HoodieAvroDataBlock;
import org.apache.hudi.common.table.log.block.HoodieDataBlock;
import org.apache.hudi.common.table.log.block.HoodieLogBlock;
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
import org.apache.hudi.common.table.timeline.HoodieInstant;
@@ -356,20 +356,7 @@ public class TableSchemaResolver {
* @return
*/
public MessageType readSchemaFromLogFile(Path path) throws IOException {
FileSystem fs = metaClient.getRawFs();
Reader reader = HoodieLogFormat.newReader(fs, new HoodieLogFile(path), null);
HoodieAvroDataBlock lastBlock = null;
while (reader.hasNext()) {
HoodieLogBlock block = reader.next();
if (block instanceof HoodieAvroDataBlock) {
lastBlock = (HoodieAvroDataBlock) block;
}
}
reader.close();
if (lastBlock != null) {
return new AvroSchemaConverter().convert(lastBlock.getSchema());
}
return null;
return readSchemaFromLogFile(metaClient.getRawFs(), path);
}
/**
@@ -394,11 +381,11 @@ public class TableSchemaResolver {
*/
public static MessageType readSchemaFromLogFile(FileSystem fs, Path path) throws IOException {
Reader reader = HoodieLogFormat.newReader(fs, new HoodieLogFile(path), null);
HoodieAvroDataBlock lastBlock = null;
HoodieDataBlock lastBlock = null;
while (reader.hasNext()) {
HoodieLogBlock block = reader.next();
if (block instanceof HoodieAvroDataBlock) {
lastBlock = (HoodieAvroDataBlock) block;
if (block instanceof HoodieDataBlock) {
lastBlock = (HoodieDataBlock) block;
}
}
reader.close();

View File

@@ -25,6 +25,7 @@ import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.log.block.HoodieAvroDataBlock;
import org.apache.hudi.common.table.log.block.HoodieCommandBlock;
import org.apache.hudi.common.table.log.block.HoodieDataBlock;
import org.apache.hudi.common.table.log.block.HoodieDeleteBlock;
import org.apache.hudi.common.table.log.block.HoodieLogBlock;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
@@ -267,7 +268,7 @@ public abstract class AbstractHoodieLogRecordScanner {
* Iterate over the GenericRecord in the block, read the hoodie key and partition path and call subclass processors to
* handle it.
*/
private void processAvroDataBlock(HoodieAvroDataBlock dataBlock) throws Exception {
private void processDataBlock(HoodieDataBlock dataBlock) throws Exception {
// TODO (NA) - Implement getRecordItr() in HoodieAvroDataBlock and use that here
List<IndexedRecord> recs = dataBlock.getRecords();
totalLogRecords.addAndGet(recs.size());
@@ -302,7 +303,7 @@ public abstract class AbstractHoodieLogRecordScanner {
HoodieLogBlock lastBlock = lastBlocks.pollLast();
switch (lastBlock.getBlockType()) {
case AVRO_DATA_BLOCK:
processAvroDataBlock((HoodieAvroDataBlock) lastBlock);
processDataBlock((HoodieAvroDataBlock) lastBlock);
break;
case DELETE_BLOCK:
Arrays.stream(((HoodieDeleteBlock) lastBlock).getKeysToDelete()).forEach(this::processNextDeletedKey);

View File

@@ -193,7 +193,7 @@ public class HoodieLogFileReader implements HoodieLogFormat.Reader {
if (nextBlockVersion.getVersion() == HoodieLogFormatVersion.DEFAULT_VERSION) {
return HoodieAvroDataBlock.getBlock(content, readerSchema);
} else {
return HoodieAvroDataBlock.getBlock(logFile, inputStream, Option.ofNullable(content), readBlockLazily,
return new HoodieAvroDataBlock(logFile, inputStream, Option.ofNullable(content), readBlockLazily,
contentPosition, contentLength, blockEndPos, readerSchema, header, footer);
}
case DELETE_BLOCK:

View File

@@ -22,7 +22,7 @@ import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.HoodieLogFile;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.log.HoodieLogFormat.Reader;
import org.apache.hudi.common.table.log.block.HoodieAvroDataBlock;
import org.apache.hudi.common.table.log.block.HoodieDataBlock;
import org.apache.hudi.common.table.log.block.HoodieLogBlock;
import org.apache.hudi.common.table.log.block.HoodieLogBlock.HeaderMetadataType;
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
@@ -49,8 +49,8 @@ public class LogReaderUtils {
HoodieTimeline completedTimeline = activeTimeline.getCommitsTimeline().filterCompletedInstants();
while (reader.hasPrev()) {
HoodieLogBlock block = reader.prev();
if (block instanceof HoodieAvroDataBlock) {
HoodieAvroDataBlock lastBlock = (HoodieAvroDataBlock) block;
if (block instanceof HoodieDataBlock) {
HoodieDataBlock lastBlock = (HoodieDataBlock) block;
if (completedTimeline
.containsOrBeforeTimelineStarts(lastBlock.getLogBlockHeader().get(HeaderMetadataType.INSTANT_TIME))) {
writerSchema = new Schema.Parser().parse(lastBlock.getLogBlockHeader().get(HeaderMetadataType.SCHEMA));

View File

@@ -36,8 +36,6 @@ import org.apache.avro.io.Encoder;
import org.apache.avro.io.EncoderFactory;
import org.apache.hadoop.fs.FSDataInputStream;
import javax.annotation.Nonnull;
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.DataInputStream;
@@ -49,56 +47,42 @@ import java.util.Iterator;
import java.util.List;
import java.util.Map;
/**
* DataBlock contains a list of records serialized using Avro. The Datablock contains 1. Data Block version 2. Total
* number of records in the block 3. Size of a record 4. Actual avro serialized content of the record
*/
public class HoodieAvroDataBlock extends HoodieLogBlock {
import javax.annotation.Nonnull;
/**
* HoodieAvroDataBlock contains a list of records serialized using Avro. It is used with the Parquet base file format.
*/
public class HoodieAvroDataBlock extends HoodieDataBlock {
private List<IndexedRecord> records;
private Schema schema;
private ThreadLocal<BinaryEncoder> encoderCache = new ThreadLocal<>();
private ThreadLocal<BinaryDecoder> decoderCache = new ThreadLocal<>();
public HoodieAvroDataBlock(@Nonnull List<IndexedRecord> records, @Nonnull Map<HeaderMetadataType, String> header,
@Nonnull Map<HeaderMetadataType, String> footer) {
super(header, footer, Option.empty(), Option.empty(), null, false);
this.records = records;
this.schema = new Schema.Parser().parse(super.getLogBlockHeader().get(HeaderMetadataType.SCHEMA));
public HoodieAvroDataBlock(@Nonnull Map<HeaderMetadataType, String> logBlockHeader,
@Nonnull Map<HeaderMetadataType, String> logBlockFooter,
@Nonnull Option<HoodieLogBlockContentLocation> blockContentLocation, @Nonnull Option<byte[]> content,
FSDataInputStream inputStream, boolean readBlockLazily) {
super(logBlockHeader, logBlockFooter, blockContentLocation, content, inputStream, readBlockLazily);
}
public HoodieAvroDataBlock(HoodieLogFile logFile, FSDataInputStream inputStream, Option<byte[]> content,
boolean readBlockLazily, long position, long blockSize, long blockEndpos, Schema readerSchema,
Map<HeaderMetadataType, String> header, Map<HeaderMetadataType, String> footer) {
super(content, inputStream, readBlockLazily,
Option.of(new HoodieLogBlockContentLocation(logFile, position, blockSize, blockEndpos)), readerSchema, header,
footer);
}
public HoodieAvroDataBlock(@Nonnull List<IndexedRecord> records, @Nonnull Map<HeaderMetadataType, String> header) {
this(records, header, new HashMap<>());
}
private HoodieAvroDataBlock(Option<byte[]> content, @Nonnull FSDataInputStream inputStream, boolean readBlockLazily,
Option<HoodieLogBlockContentLocation> blockContentLocation, Schema readerSchema,
@Nonnull Map<HeaderMetadataType, String> headers, @Nonnull Map<HeaderMetadataType, String> footer) {
super(headers, footer, blockContentLocation, content, inputStream, readBlockLazily);
this.schema = readerSchema;
}
public static HoodieLogBlock getBlock(HoodieLogFile logFile, FSDataInputStream inputStream, Option<byte[]> content,
boolean readBlockLazily, long position, long blockSize, long blockEndpos, Schema readerSchema,
Map<HeaderMetadataType, String> header, Map<HeaderMetadataType, String> footer) {
return new HoodieAvroDataBlock(content, inputStream, readBlockLazily,
Option.of(new HoodieLogBlockContentLocation(logFile, position, blockSize, blockEndpos)), readerSchema, header,
footer);
super(records, header, new HashMap<>());
}
@Override
public byte[] getContentBytes() throws IOException {
// In case this method is called before realizing records from content
if (getContent().isPresent()) {
return getContent().get();
} else if (readBlockLazily && !getContent().isPresent() && records == null) {
// read block lazily
createRecordsFromContentBytes();
}
public HoodieLogBlockType getBlockType() {
return HoodieLogBlockType.AVRO_DATA_BLOCK;
}
@Override
protected byte[] serializeRecords() throws IOException {
Schema schema = new Schema.Parser().parse(super.getLogBlockHeader().get(HeaderMetadataType.SCHEMA));
GenericDatumWriter<IndexedRecord> writer = new GenericDatumWriter<>(schema);
ByteArrayOutputStream baos = new ByteArrayOutputStream();
@@ -137,40 +121,10 @@ public class HoodieAvroDataBlock extends HoodieLogBlock {
return baos.toByteArray();
}
@Override
public HoodieLogBlockType getBlockType() {
return HoodieLogBlockType.AVRO_DATA_BLOCK;
}
public List<IndexedRecord> getRecords() {
if (records == null) {
try {
// in case records are absent, read content lazily and then convert to IndexedRecords
createRecordsFromContentBytes();
} catch (IOException io) {
throw new HoodieIOException("Unable to convert content bytes to records", io);
}
}
return records;
}
public Schema getSchema() {
// if getSchema was invoked before converting byte [] to records
if (records == null) {
getRecords();
}
return schema;
}
// TODO (na) - Break down content into smaller chunks of byte [] to be GC as they are used
// TODO (na) - Implement a recordItr instead of recordList
private void createRecordsFromContentBytes() throws IOException {
if (readBlockLazily && !getContent().isPresent()) {
// read log block contents from disk
inflate();
}
@Override
protected void deserializeRecords() throws IOException {
SizeAwareDataInputStream dis =
new SizeAwareDataInputStream(new DataInputStream(new ByteArrayInputStream(getContent().get())));
@@ -212,6 +166,9 @@ public class HoodieAvroDataBlock extends HoodieLogBlock {
//----------------------------------------------------------------------------------------
// DEPRECATED METHODS
//
// These methods were only supported by HoodieAvroDataBlock and have been deprecated. Hence,
// these are only implemented here even though they duplicate the code from HoodieAvroDataBlock.
//----------------------------------------------------------------------------------------
/**
@@ -230,7 +187,7 @@ public class HoodieAvroDataBlock extends HoodieLogBlock {
* HoodieLogFormat V1.
*/
@Deprecated
public static HoodieLogBlock getBlock(byte[] content, Schema readerSchema) throws IOException {
public static HoodieAvroDataBlock getBlock(byte[] content, Schema readerSchema) throws IOException {
SizeAwareDataInputStream dis = new SizeAwareDataInputStream(new DataInputStream(new ByteArrayInputStream(content)));
@@ -302,5 +259,4 @@ public class HoodieAvroDataBlock extends HoodieLogBlock {
output.close();
return baos.toByteArray();
}
}

View File

@@ -0,0 +1,132 @@
/*
* 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.table.log.block;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.avro.Schema;
import org.apache.avro.generic.IndexedRecord;
import org.apache.hadoop.fs.FSDataInputStream;
import javax.annotation.Nonnull;
import java.io.IOException;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
/**
* DataBlock contains a list of records serialized using formats compatible with the base file format.
* For each base file format there is a corresponding DataBlock format.
*
* The Datablock contains:
* 1. Data Block version
* 2. Total number of records in the block
* 3. Actual serialized content of the records
*/
public abstract class HoodieDataBlock extends HoodieLogBlock {
protected List<IndexedRecord> records;
protected Schema schema;
public HoodieDataBlock(@Nonnull Map<HeaderMetadataType, String> logBlockHeader,
@Nonnull Map<HeaderMetadataType, String> logBlockFooter,
@Nonnull Option<HoodieLogBlockContentLocation> blockContentLocation, @Nonnull Option<byte[]> content,
FSDataInputStream inputStream, boolean readBlockLazily) {
super(logBlockHeader, logBlockFooter, blockContentLocation, content, inputStream, readBlockLazily);
}
public HoodieDataBlock(@Nonnull List<IndexedRecord> records, @Nonnull Map<HeaderMetadataType, String> header,
@Nonnull Map<HeaderMetadataType, String> footer) {
super(header, footer, Option.empty(), Option.empty(), null, false);
this.records = records;
this.schema = new Schema.Parser().parse(super.getLogBlockHeader().get(HeaderMetadataType.SCHEMA));
}
public HoodieDataBlock(@Nonnull List<IndexedRecord> records, @Nonnull Map<HeaderMetadataType, String> header) {
this(records, header, new HashMap<>());
}
protected HoodieDataBlock(Option<byte[]> content, @Nonnull FSDataInputStream inputStream, boolean readBlockLazily,
Option<HoodieLogBlockContentLocation> blockContentLocation, Schema readerSchema,
@Nonnull Map<HeaderMetadataType, String> headers, @Nonnull Map<HeaderMetadataType, String> footer) {
super(headers, footer, blockContentLocation, content, inputStream, readBlockLazily);
this.schema = readerSchema;
}
public static HoodieLogBlock getBlock(HoodieLogBlockType logDataBlockFormat, List<IndexedRecord> recordList,
Map<HeaderMetadataType, String> header) {
switch (logDataBlockFormat) {
case AVRO_DATA_BLOCK:
return new HoodieAvroDataBlock(recordList, header);
default:
throw new HoodieException("Data block format " + logDataBlockFormat + " not implemented");
}
}
@Override
public byte[] getContentBytes() throws IOException {
// In case this method is called before realizing records from content
if (getContent().isPresent()) {
return getContent().get();
} else if (readBlockLazily && !getContent().isPresent() && records == null) {
// read block lazily
createRecordsFromContentBytes();
}
return serializeRecords();
}
public abstract HoodieLogBlockType getBlockType();
public List<IndexedRecord> getRecords() {
if (records == null) {
try {
// in case records are absent, read content lazily and then convert to IndexedRecords
createRecordsFromContentBytes();
} catch (IOException io) {
throw new HoodieIOException("Unable to convert content bytes to records", io);
}
}
return records;
}
public Schema getSchema() {
// if getSchema was invoked before converting byte [] to records
if (records == null) {
getRecords();
}
return schema;
}
private void createRecordsFromContentBytes() throws IOException {
if (readBlockLazily && !getContent().isPresent()) {
// read log block contents from disk
inflate();
}
deserializeRecords();
}
protected abstract byte[] serializeRecords() throws IOException;
protected abstract void deserializeRecords() throws IOException;
}

View File

@@ -84,9 +84,7 @@ public abstract class HoodieLogBlock {
throw new HoodieException("No implementation was provided");
}
public HoodieLogBlockType getBlockType() {
throw new HoodieException("No implementation was provided");
}
public abstract HoodieLogBlockType getBlockType();
public long getLogBlockLength() {
throw new HoodieException("No implementation was provided");

View File

@@ -121,7 +121,8 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV
}
});
long storePartitionsTs = timer.endTimer();
LOG.info("addFilesToView: NumFiles=" + statuses.length + ", FileGroupsCreationTime=" + fgBuildTimeTakenMs
LOG.info("addFilesToView: NumFiles=" + statuses.length + ", NumFileGroups=" + fileGroups.size()
+ ", FileGroupsCreationTime=" + fgBuildTimeTakenMs
+ ", StoreTimeTaken=" + storePartitionsTs);
return fileGroups;
}

View File

@@ -0,0 +1,77 @@
/*
* 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 org.apache.hudi.common.util.queue.BoundedInMemoryQueue;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.parquet.hadoop.ParquetReader;
import java.io.IOException;
import java.util.Iterator;
/**
* This class wraps a parquet reader and provides an iterator based api to read from a parquet file. This is used in
* {@link BoundedInMemoryQueue}
*/
public class ParquetReaderIterator<T> implements Iterator<T> {
// Parquet reader for an existing parquet file
private final ParquetReader<T> parquetReader;
// Holds the next entry returned by the parquet reader
private T next;
public ParquetReaderIterator(ParquetReader<T> parquetReader) {
this.parquetReader = parquetReader;
}
@Override
public boolean hasNext() {
try {
// To handle when hasNext() is called multiple times for idempotency and/or the first time
if (this.next == null) {
this.next = parquetReader.read();
}
return this.next != null;
} catch (IOException io) {
throw new HoodieIOException("unable to read next record from parquet file ", io);
}
}
@Override
public T next() {
try {
// To handle case when next() is called before hasNext()
if (this.next == null) {
if (!hasNext()) {
throw new HoodieIOException("No more records left to read from parquet file");
}
}
T retVal = this.next;
this.next = parquetReader.read();
return retVal;
} catch (IOException io) {
throw new HoodieIOException("unable to read next record from parquet file ", io);
}
}
public void close() throws IOException {
parquetReader.close();
}
}

View File

@@ -0,0 +1,44 @@
/*
* 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.io.storage;
import java.io.IOException;
import java.util.Iterator;
import java.util.Set;
import org.apache.avro.Schema;
import org.apache.avro.generic.IndexedRecord;
import org.apache.hudi.common.bloom.BloomFilter;
public interface HoodieFileReader<R extends IndexedRecord> {
public String[] readMinMaxRecordKeys();
public BloomFilter readBloomFilter();
public Set<String> filterRowKeys(Set<String> candidateRowKeys);
public Iterator<R> getRecordIterator(Schema schema) throws IOException;
Schema getSchema();
void close();
long getTotalRecords();
}

View File

@@ -0,0 +1,47 @@
/*
* 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.io.storage;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.avro.generic.IndexedRecord;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import java.io.IOException;
import static org.apache.hudi.common.model.HoodieFileFormat.PARQUET;
public class HoodieFileReaderFactory {
public static <T extends HoodieRecordPayload, R extends IndexedRecord> HoodieFileReader<R> getFileReader(
Configuration conf, Path path) throws IOException {
final String extension = FSUtils.getFileExtension(path.toString());
if (PARQUET.getFileExtension().equals(extension)) {
return newParquetFileReader(conf, path);
}
throw new UnsupportedOperationException(extension + " format not supported yet.");
}
private static <T extends HoodieRecordPayload, R extends IndexedRecord> HoodieFileReader<R> newParquetFileReader(
Configuration conf, Path path) throws IOException {
return new HoodieParquetReader<>(conf, path);
}
}

View File

@@ -0,0 +1,80 @@
/*
* 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.io.storage;
import java.io.IOException;
import java.util.Iterator;
import java.util.Set;
import org.apache.avro.Schema;
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.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;
public class HoodieParquetReader<R extends IndexedRecord> implements HoodieFileReader {
private Path path;
private Configuration conf;
public HoodieParquetReader(Configuration configuration, Path path) {
this.conf = configuration;
this.path = path;
}
public String[] readMinMaxRecordKeys() {
return ParquetUtils.readMinMaxRecordKeys(conf, path);
}
@Override
public BloomFilter readBloomFilter() {
return ParquetUtils.readBloomFilterFromParquetMetadata(conf, path);
}
@Override
public Set<String> filterRowKeys(Set candidateRowKeys) {
return ParquetUtils.filterParquetRowKeys(conf, path, candidateRowKeys);
}
@Override
public Iterator<R> getRecordIterator(Schema schema) throws IOException {
AvroReadSupport.setAvroReadSchema(conf, schema);
ParquetReader<IndexedRecord> reader = AvroParquetReader.<IndexedRecord>builder(path).withConf(conf).build();
return new ParquetReaderIterator(reader);
}
@Override
public Schema getSchema() {
return ParquetUtils.readAvroSchema(conf, path);
}
@Override
public void close() {
}
@Override
public long getTotalRecords() {
// TODO Auto-generated method stub
return 0;
}
}