Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -360,7 +360,7 @@ protected void appendDataAndDeleteBlocks(Map<HeaderMetadataType, String> header)
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, writeSchemaWithMetaFields.toString());
List<HoodieLogBlock> blocks = new ArrayList<>(2);
if (recordList.size() > 0) {
blocks.add(HoodieDataBlock.getBlock(hoodieTable.getLogDataBlockFormat(), recordList, header));
blocks.add(HoodieDataBlock.getBlock(hoodieTable.getLogDataBlockType(), recordList, header));
}
if (keysToDelete.size() > 0) {
blocks.add(new HoodieDeleteBlock(keysToDelete.toArray(new HoodieKey[keysToDelete.size()]), header));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -44,6 +44,7 @@
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;
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;
Expand Down Expand Up @@ -702,16 +703,21 @@ public HoodieFileFormat getLogFileFormat() {
return metaClient.getTableConfig().getLogFileFormat();
}

public HoodieLogBlockType getLogDataBlockFormat() {
switch (getBaseFileFormat()) {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think this config is not required.

For a dataset with a PARQUET file format, an inline log block format of PARQUET makes sense. Mix and match of base file format and inline log format does not seem to make writing data or query engine side faster.

So we can have a config - useInlineFiles() - when enabled, the inline file format will be format and it will have the same format as the base file format. This needs to be fixed for the dataset so can be from table config.

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 HoodieLogBlockType getLogDataBlockType() {
HoodieLogBlock.HoodieLogBlockType logBlockType = metaClient.getTableConfig().getLogBlockFormat();
if (logBlockType != null) {
return logBlockType;
} else {
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 type");
}
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@
import org.apache.hudi.common.model.HoodieFileFormat;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload;
import org.apache.hudi.common.table.log.block.HoodieLogBlock;
import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.ValidationUtils;
Expand Down Expand Up @@ -120,6 +121,11 @@ public class HoodieTableConfig extends HoodieConfig {
.withAlternatives("hoodie.table.rt.file.format")
.withDocumentation("Log format used for the delta logs.");

public static final ConfigProperty<String> LOG_BLOCK_TYPE = ConfigProperty
.key("hoodie.table.log.block.type")
.noDefaultValue()
.withDocumentation("Log block type used for the delta logs.");

public static final ConfigProperty<String> TIMELINE_LAYOUT_VERSION = ConfigProperty
.key("hoodie.timeline.layout.version")
.noDefaultValue()
Expand Down Expand Up @@ -346,6 +352,20 @@ public HoodieFileFormat getLogFileFormat() {
return HoodieFileFormat.valueOf(getStringOrDefault(LOG_FILE_FORMAT));
}

/**
* Get the log block Format.
*
* @return HoodieBlockFormat for the log block
*/
public HoodieLogBlock.HoodieLogBlockType getLogBlockFormat() {
String logBlockTypeConfig = getString(LOG_BLOCK_TYPE);
if (logBlockTypeConfig != null) {
return HoodieLogBlock.HoodieLogBlockType.valueOf(getStringOrDefault(LOG_BLOCK_TYPE));
} else {
return null;
}
}

/**
* Get the relative path of archive log folder under metafolder, for this table.
*/
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@
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.HoodieParquetDataBlock;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.SpillableMapUtils;
Expand Down Expand Up @@ -204,6 +205,7 @@ public void scan(Option<List<String>> keys) {
switch (logBlock.getBlockType()) {
case HFILE_DATA_BLOCK:
case AVRO_DATA_BLOCK:
case PARQUET_DATA_BLOCK:
LOG.info("Reading a data block from file " + logFile.getPath() + " at instant "
+ logBlock.getLogBlockHeader().get(INSTANT_TIME));
if (isNewInstantBlock(logBlock) && !readBlocksLazily) {
Expand Down Expand Up @@ -381,6 +383,9 @@ private void processQueuedBlocksForInstant(Deque<HoodieLogBlock> logBlocks, int
case HFILE_DATA_BLOCK:
processDataBlock((HoodieHFileDataBlock) lastBlock, keys);
break;
case PARQUET_DATA_BLOCK:
processDataBlock((HoodieParquetDataBlock) lastBlock, keys);
break;
case DELETE_BLOCK:
Arrays.stream(((HoodieDeleteBlock) lastBlock).getKeysToDelete()).forEach(this::processNextDeletedKey);
break;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@
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.HoodieLogBlock.HoodieLogBlockType;
import org.apache.hudi.common.table.log.block.HoodieParquetDataBlock;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.ValidationUtils;
import org.apache.hudi.exception.CorruptedLogFileException;
Expand Down Expand Up @@ -256,6 +257,9 @@ private HoodieLogBlock readBlock() throws IOException {
case HFILE_DATA_BLOCK:
return new HoodieHFileDataBlock(logFile, inputStream, Option.ofNullable(content), readBlockLazily,
contentPosition, contentLength, blockEndPos, readerSchema, header, footer, enableInlineReading);
case PARQUET_DATA_BLOCK:
return new HoodieParquetDataBlock(logFile, inputStream, Option.ofNullable(content), readBlockLazily,
contentPosition, contentLength, blockEndPos, readerSchema, header, footer);
case DELETE_BLOCK:
return HoodieDeleteBlock.getBlock(logFile, inputStream, Option.ofNullable(content), readBlockLazily,
contentPosition, contentLength, blockEndPos, header, footer);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -79,6 +79,8 @@ public static HoodieLogBlock getBlock(HoodieLogBlockType logDataBlockFormat, Lis
return new HoodieAvroDataBlock(recordList, header);
case HFILE_DATA_BLOCK:
return new HoodieHFileDataBlock(recordList, header);
case PARQUET_DATA_BLOCK:
return new HoodieParquetDataBlock(recordList, header);
default:
throw new HoodieException("Data block format " + logDataBlockFormat + " not implemented");
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -109,7 +109,7 @@ public Option<byte[]> getContent() {
* Type of the log block WARNING: This enum is serialized as the ordinal. Only add new enums at the end.
*/
public enum HoodieLogBlockType {
COMMAND_BLOCK, DELETE_BLOCK, CORRUPT_BLOCK, AVRO_DATA_BLOCK, HFILE_DATA_BLOCK
COMMAND_BLOCK, DELETE_BLOCK, CORRUPT_BLOCK, AVRO_DATA_BLOCK, HFILE_DATA_BLOCK, PARQUET_DATA_BLOCK
}

/**
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,168 @@
/*
* 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.avro.HoodieAvroWriteSupport;
import org.apache.hudi.common.bloom.BloomFilter;
import org.apache.hudi.common.bloom.BloomFilterFactory;
import org.apache.hudi.common.bloom.BloomFilterTypeCode;
import org.apache.hudi.common.fs.inline.InLineFSUtils;
import org.apache.hudi.common.fs.inline.InLineFileSystem;
import org.apache.hudi.common.model.HoodieLogFile;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.hudi.io.storage.HoodieAvroParquetConfig;
import org.apache.hudi.io.storage.HoodieParquetReader;
import org.apache.hudi.io.storage.HoodieParquetStreamWriter;

import org.apache.avro.Schema;
import org.apache.avro.generic.IndexedRecord;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.Path;
import org.apache.parquet.avro.AvroSchemaConverter;
import org.apache.parquet.hadoop.ParquetWriter;
import org.apache.parquet.hadoop.metadata.CompressionCodecName;

import javax.annotation.Nonnull;

import java.io.ByteArrayOutputStream;
import java.io.DataOutputStream;
import java.io.IOException;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;

/**
* HoodieParquetDataBlock contains a list of records serialized using Parquet.
*/
public class HoodieParquetDataBlock extends HoodieDataBlock {

public HoodieParquetDataBlock(@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 HoodieParquetDataBlock(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 HoodieParquetDataBlock(@Nonnull List<IndexedRecord> records, @Nonnull Map<HeaderMetadataType, String> header) {
super(records, header, new HashMap<>());
}

@Override
public HoodieLogBlockType getBlockType() {
return HoodieLogBlockType.PARQUET_DATA_BLOCK;
}

@Override
protected byte[] serializeRecords() throws IOException {
// TODO: Need to decide from where to fetch all config values required below. We can't re-use index config as the purpose is different.
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think the configs here should match the configs for the dataset base file format.

Under what case would the user want separate configs for base file and inline log blocks?

// And these are very specific to data blocks. Once we have consensus, we might need to route them to log block constructors (
// as of now, log block constructors does not take in any configs in general).
BloomFilter filter = BloomFilterFactory.createBloomFilter(
Integer.parseInt("60000"),//HoodieIndexConfig.BLOOM_FILTER_NUM_ENTRIES.defaultValue()),
Double.parseDouble("0.000000001"),//HoodieIndexConfig.BLOOM_FILTER_FPP.defaultValue()),
Integer.parseInt("100000"),//HoodieIndexConfig.HOODIE_BLOOM_INDEX_FILTER_DYNAMIC_MAX_ENTRIES.defaultValue()),
BloomFilterTypeCode.SIMPLE.name());//HoodieIndexConfig.BLOOM_INDEX_FILTER_TYPE.defaultValue());

HoodieAvroWriteSupport writeSupport = new HoodieAvroWriteSupport(
new AvroSchemaConverter().convert(schema), schema, Option.of(filter));

HoodieAvroParquetConfig avroParquetConfig = new HoodieAvroParquetConfig(writeSupport, CompressionCodecName.GZIP,
ParquetWriter.DEFAULT_BLOCK_SIZE, ParquetWriter.DEFAULT_PAGE_SIZE, 1024 * 1024 * 1024,
new Configuration(), Double.parseDouble(String.valueOf(0.1)));//HoodieStorageConfig.PARQUET_COMPRESSION_RATIO.defaultValue()));

ByteArrayOutputStream baos = new ByteArrayOutputStream();
DataOutputStream outputStream = null;
HoodieParquetStreamWriter<IndexedRecord> parquetWriter = null;
try {
outputStream = new DataOutputStream(baos);
parquetWriter = new HoodieParquetStreamWriter<>(outputStream, avroParquetConfig);
Iterator<IndexedRecord> itr = records.iterator();
if (records.size() > 0) {
Schema.Field keyField = records.get(0).getSchema().getField(HoodieRecord.RECORD_KEY_METADATA_FIELD);
if (keyField == null) {
throw new HoodieIOException("Record key field missing from schema for records to be written to Parquet data block");
}
while (itr.hasNext()) {
IndexedRecord record = itr.next();
String recordKey = record.get(keyField.pos()).toString();
parquetWriter.writeAvro(recordKey, record);
}
outputStream.flush();
}
} finally {
if (outputStream != null) {
outputStream.close();
}
if (parquetWriter != null) {
parquetWriter.close();
}
}

return baos.toByteArray();
}

@Override
public List<IndexedRecord> getRecords() {
try {
records = new ArrayList<>();
// Get schema from the header
Schema writerSchema = new Schema.Parser().parse(super.getLogBlockHeader().get(HeaderMetadataType.SCHEMA));
// If readerSchema was not present, use writerSchema
if (schema == null) {
schema = writerSchema;
}
Configuration conf = new Configuration();
Configuration inlineConf = new Configuration();
inlineConf.set("fs." + InLineFileSystem.SCHEME + ".impl", InLineFileSystem.class.getName());

Path inlinePath = InLineFSUtils.getInlineFilePath(
getBlockContentLocation().get().getLogFile().getPath(),
getBlockContentLocation().get().getLogFile().getPath().getFileSystem(conf).getScheme(),
getBlockContentLocation().get().getContentPositionInLogFile(),
getBlockContentLocation().get().getBlockSize());

HoodieParquetReader<IndexedRecord> parquetReader = new HoodieParquetReader<>(inlineConf, inlinePath);
Iterator<IndexedRecord> recordIterator = parquetReader.getRecordIterator(schema);
while (recordIterator.hasNext()) {
records.add(recordIterator.next());
}
return records;
} catch (IOException e) {
throw new HoodieIOException("Reading parquet inlining failed ", e);
}
}

@Override
protected void deserializeRecords() throws IOException {
throw new IOException("Not implemented");
}
}
Loading