1
0

[HUDI-431] Adding support for Parquet in MOR LogBlocks (#4333)

- Adding support for Parquet in MOR tables Log blocks

Co-authored-by: Sivabalan Narayanan <n.siva.b@gmail.com>
This commit is contained in:
Alexey Kudinkin
2022-02-02 11:35:05 -08:00
committed by GitHub
parent caef3d5c58
commit a68e1dc2db
45 changed files with 1613 additions and 670 deletions

View File

@@ -83,14 +83,17 @@ public class HoodieStorageConfig extends HoodieConfig {
.withDocumentation("Lower values increase the size of metadata tracked within HFile, but can offer potentially "
+ "faster lookup times.");
// used to size log files
public static final ConfigProperty<String> LOGFILE_DATA_BLOCK_FORMAT = ConfigProperty
.key("hoodie.logfile.data.block.format")
.noDefaultValue()
.withDocumentation("Format of the data block within delta logs. Following formats are currently supported \"avro\", \"hfile\", \"parquet\"");
public static final ConfigProperty<String> LOGFILE_MAX_SIZE = ConfigProperty
.key("hoodie.logfile.max.size")
.defaultValue(String.valueOf(1024 * 1024 * 1024)) // 1 GB
.withDocumentation("LogFile max size. This is the maximum size allowed for a log file "
+ "before it is rolled over to the next version.");
// used to size data blocks in log file
public static final ConfigProperty<String> LOGFILE_DATA_BLOCK_MAX_SIZE = ConfigProperty
.key("hoodie.logfile.data.block.max.size")
.defaultValue(String.valueOf(256 * 1024 * 1024))

View File

@@ -37,9 +37,11 @@ import org.apache.hudi.common.model.HoodieFileFormat;
import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload;
import org.apache.hudi.common.model.WriteConcurrencyMode;
import org.apache.hudi.common.table.HoodieTableConfig;
import org.apache.hudi.common.table.log.block.HoodieLogBlock;
import org.apache.hudi.common.table.marker.MarkerType;
import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion;
import org.apache.hudi.common.table.view.FileSystemViewStorageConfig;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.ReflectionUtils;
import org.apache.hudi.common.util.ValidationUtils;
import org.apache.hudi.config.metrics.HoodieMetricsConfig;
@@ -1506,6 +1508,11 @@ public class HoodieWriteConfig extends HoodieConfig {
return getString(HoodieStorageConfig.PARQUET_OUTPUT_TIMESTAMP_TYPE);
}
public Option<HoodieLogBlock.HoodieLogBlockType> getLogDataBlockFormat() {
return Option.ofNullable(getString(HoodieStorageConfig.LOGFILE_DATA_BLOCK_FORMAT))
.map(HoodieLogBlock.HoodieLogBlockType::fromId);
}
public long getLogFileMaxSize() {
return getLong(HoodieStorageConfig.LOGFILE_MAX_SIZE);
}

View File

@@ -38,10 +38,12 @@ import org.apache.hudi.common.model.IOType;
import org.apache.hudi.common.table.log.AppendResult;
import org.apache.hudi.common.table.log.HoodieLogFormat;
import org.apache.hudi.common.table.log.HoodieLogFormat.Writer;
import org.apache.hudi.common.table.log.block.HoodieDataBlock;
import org.apache.hudi.common.table.log.block.HoodieAvroDataBlock;
import org.apache.hudi.common.table.log.block.HoodieDeleteBlock;
import org.apache.hudi.common.table.log.block.HoodieHFileDataBlock;
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.log.block.HoodieParquetDataBlock;
import org.apache.hudi.common.table.view.TableFileSystemView.SliceView;
import org.apache.hudi.common.util.DefaultSizeEstimator;
import org.apache.hudi.common.util.Option;
@@ -49,6 +51,7 @@ import org.apache.hudi.common.util.ReflectionUtils;
import org.apache.hudi.common.util.SizeEstimator;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieAppendException;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.exception.HoodieUpsertException;
import org.apache.hudi.table.HoodieTable;
@@ -360,13 +363,13 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload, I, K, O> extends
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, writeSchemaWithMetaFields.toString());
List<HoodieLogBlock> blocks = new ArrayList<>(2);
if (recordList.size() > 0) {
if (config.populateMetaFields()) {
blocks.add(HoodieDataBlock.getBlock(hoodieTable.getLogDataBlockFormat(), recordList, header));
} else {
final String keyField = hoodieTable.getMetaClient().getTableConfig().getRecordKeyFieldProp();
blocks.add(HoodieDataBlock.getBlock(hoodieTable.getLogDataBlockFormat(), recordList, header, keyField));
}
String keyField = config.populateMetaFields()
? HoodieRecord.RECORD_KEY_METADATA_FIELD
: hoodieTable.getMetaClient().getTableConfig().getRecordKeyFieldProp();
blocks.add(getBlock(config, pickLogDataBlockFormat(), recordList, header, keyField));
}
if (keysToDelete.size() > 0) {
blocks.add(new HoodieDeleteBlock(keysToDelete.toArray(new HoodieKey[keysToDelete.size()]), header));
}
@@ -497,4 +500,40 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload, I, K, O> extends
numberOfRecords = 0;
}
}
private HoodieLogBlock.HoodieLogBlockType pickLogDataBlockFormat() {
Option<HoodieLogBlock.HoodieLogBlockType> logBlockTypeOpt = config.getLogDataBlockFormat();
if (logBlockTypeOpt.isPresent()) {
return logBlockTypeOpt.get();
}
// Fallback to deduce data-block type based on the base file format
switch (hoodieTable.getBaseFileFormat()) {
case PARQUET:
case ORC:
return HoodieLogBlock.HoodieLogBlockType.AVRO_DATA_BLOCK;
case HFILE:
return HoodieLogBlock.HoodieLogBlockType.HFILE_DATA_BLOCK;
default:
throw new HoodieException("Base file format " + hoodieTable.getBaseFileFormat()
+ " does not have associated log block type");
}
}
private static HoodieLogBlock getBlock(HoodieWriteConfig writeConfig,
HoodieLogBlock.HoodieLogBlockType logDataBlockFormat,
List<IndexedRecord> recordList,
Map<HeaderMetadataType, String> header,
String keyField) {
switch (logDataBlockFormat) {
case AVRO_DATA_BLOCK:
return new HoodieAvroDataBlock(recordList, header, keyField);
case HFILE_DATA_BLOCK:
return new HoodieHFileDataBlock(recordList, header, writeConfig.getHFileCompressionAlgorithm());
case PARQUET_DATA_BLOCK:
return new HoodieParquetDataBlock(recordList, header, keyField, writeConfig.getParquetCompressionCodec());
default:
throw new HoodieException("Data block format " + logDataBlockFormat + " not implemented");
}
}
}

View File

@@ -1,42 +0,0 @@
/*
* 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.avro.HoodieAvroWriteSupport;
import org.apache.hadoop.conf.Configuration;
import org.apache.parquet.hadoop.metadata.CompressionCodecName;
/**
* ParquetConfig for writing avro records in Parquet files.
*/
public class HoodieAvroParquetConfig extends HoodieBaseParquetConfig<HoodieAvroWriteSupport> {
public HoodieAvroParquetConfig(HoodieAvroWriteSupport writeSupport, CompressionCodecName compressionCodecName,
int blockSize, int pageSize, long maxFileSize, Configuration hadoopConf,
double compressionRatio) {
super(writeSupport, compressionCodecName, blockSize, pageSize, maxFileSize, hadoopConf, compressionRatio);
}
public HoodieAvroParquetConfig(HoodieAvroWriteSupport writeSupport, CompressionCodecName compressionCodecName,
int blockSize, int pageSize, long maxFileSize, Configuration hadoopConf,
double compressionRatio, boolean directoryEnabled) {
super(writeSupport, compressionCodecName, blockSize, pageSize, maxFileSize, hadoopConf, compressionRatio, directoryEnabled);
}
}

View File

@@ -1,86 +0,0 @@
/*
* 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.hadoop.conf.Configuration;
import org.apache.parquet.hadoop.metadata.CompressionCodecName;
/**
* Base ParquetConfig to hold config params for writing to Parquet.
* @param <T>
*/
public class HoodieBaseParquetConfig<T> {
private final T writeSupport;
private final CompressionCodecName compressionCodecName;
private final int blockSize;
private final int pageSize;
private final long maxFileSize;
private final Configuration hadoopConf;
private final double compressionRatio;
private final boolean dictionaryEnabled;
public HoodieBaseParquetConfig(T writeSupport, CompressionCodecName compressionCodecName, int blockSize,
int pageSize, long maxFileSize, Configuration hadoopConf, double compressionRatio) {
this(writeSupport, compressionCodecName, blockSize, pageSize, maxFileSize, hadoopConf, compressionRatio, false);
}
public HoodieBaseParquetConfig(T writeSupport, CompressionCodecName compressionCodecName, int blockSize,
int pageSize, long maxFileSize, Configuration hadoopConf, double compressionRatio, boolean dictionaryEnabled) {
this.writeSupport = writeSupport;
this.compressionCodecName = compressionCodecName;
this.blockSize = blockSize;
this.pageSize = pageSize;
this.maxFileSize = maxFileSize;
this.hadoopConf = hadoopConf;
this.compressionRatio = compressionRatio;
this.dictionaryEnabled = dictionaryEnabled;
}
public CompressionCodecName getCompressionCodecName() {
return compressionCodecName;
}
public int getBlockSize() {
return blockSize;
}
public int getPageSize() {
return pageSize;
}
public long getMaxFileSize() {
return maxFileSize;
}
public Configuration getHadoopConf() {
return hadoopConf;
}
public double getCompressionRatio() {
return compressionRatio;
}
public T getWriteSupport() {
return writeSupport;
}
public boolean dictionaryEnabled() {
return dictionaryEnabled;
}
}

View File

@@ -51,13 +51,23 @@ public class HoodieParquetWriter<T extends HoodieRecordPayload, R extends Indexe
private final TaskContextSupplier taskContextSupplier;
private final boolean populateMetaFields;
public HoodieParquetWriter(String instantTime, Path file, HoodieAvroParquetConfig parquetConfig,
Schema schema, TaskContextSupplier taskContextSupplier, boolean populateMetaFields) throws IOException {
public HoodieParquetWriter(String instantTime,
Path file,
HoodieAvroParquetConfig parquetConfig,
Schema schema,
TaskContextSupplier taskContextSupplier,
boolean populateMetaFields) throws IOException {
super(HoodieWrapperFileSystem.convertToHoodiePath(file, parquetConfig.getHadoopConf()),
ParquetFileWriter.Mode.CREATE, parquetConfig.getWriteSupport(), parquetConfig.getCompressionCodecName(),
parquetConfig.getBlockSize(), parquetConfig.getPageSize(), parquetConfig.getPageSize(),
parquetConfig.dictionaryEnabled(), DEFAULT_IS_VALIDATING_ENABLED,
DEFAULT_WRITER_VERSION, FSUtils.registerFileSystem(file, parquetConfig.getHadoopConf()));
ParquetFileWriter.Mode.CREATE,
parquetConfig.getWriteSupport(),
parquetConfig.getCompressionCodecName(),
parquetConfig.getBlockSize(),
parquetConfig.getPageSize(),
parquetConfig.getPageSize(),
parquetConfig.dictionaryEnabled(),
DEFAULT_IS_VALIDATING_ENABLED,
DEFAULT_WRITER_VERSION,
FSUtils.registerFileSystem(file, parquetConfig.getHadoopConf()));
this.file = HoodieWrapperFileSystem.convertToHoodiePath(file, parquetConfig.getHadoopConf());
this.fs =
(HoodieWrapperFileSystem) this.file.getFileSystem(FSUtils.registerFileSystem(file, parquetConfig.getHadoopConf()));

View File

@@ -18,7 +18,11 @@
package org.apache.hudi.table;
import org.apache.avro.Schema;
import org.apache.avro.specific.SpecificRecordBase;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.avro.HoodieAvroUtils;
import org.apache.hudi.avro.model.HoodieCleanMetadata;
import org.apache.hudi.avro.model.HoodieCleanerPlan;
@@ -44,7 +48,6 @@ import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.model.HoodieWriteStat;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.TableSchemaResolver;
import org.apache.hudi.common.table.log.block.HoodieLogBlock.HoodieLogBlockType;
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
@@ -69,11 +72,6 @@ import org.apache.hudi.table.action.HoodieWriteMetadata;
import org.apache.hudi.table.action.bootstrap.HoodieBootstrapWriteMetadata;
import org.apache.hudi.table.marker.WriteMarkers;
import org.apache.hudi.table.marker.WriteMarkersFactory;
import org.apache.avro.Schema;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.table.storage.HoodieLayoutFactory;
import org.apache.hudi.table.storage.HoodieStorageLayout;
import org.apache.log4j.LogManager;
@@ -719,19 +717,6 @@ public abstract class HoodieTable<T extends HoodieRecordPayload, I, K, O> implem
return metaClient.getTableConfig().getLogFileFormat();
}
public HoodieLogBlockType getLogDataBlockFormat() {
switch (getBaseFileFormat()) {
case PARQUET:
case ORC:
return HoodieLogBlockType.AVRO_DATA_BLOCK;
case HFILE:
return HoodieLogBlockType.HFILE_DATA_BLOCK;
default:
throw new HoodieException("Base file format " + getBaseFileFormat()
+ " does not have associated log block format");
}
}
public String getBaseFileExtension() {
return getBaseFileFormat().getFileExtension();
}

View File

@@ -182,7 +182,7 @@ public class HoodieWriteableTestTable extends HoodieMetadataTestTable {
LOG.warn("Failed to convert record " + r.toString(), e);
return null;
}
}).collect(Collectors.toList()), header));
}).collect(Collectors.toList()), header, HoodieRecord.RECORD_KEY_METADATA_FIELD));
return Pair.of(partitionPath, logWriter.getLogFile());
}
}