Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[HUDI-764] [HUDI-765] ORC reader and writer implementation #2793

Closed
wants to merge 4 commits into from
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.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
Expand Up @@ -39,10 +39,21 @@ public class HoodieStorageConfig extends DefaultHoodieConfig {
public static final String DEFAULT_PARQUET_BLOCK_SIZE_BYTES = DEFAULT_PARQUET_FILE_MAX_BYTES;
public static final String PARQUET_PAGE_SIZE_BYTES = "hoodie.parquet.page.size";
public static final String DEFAULT_PARQUET_PAGE_SIZE_BYTES = String.valueOf(1 * 1024 * 1024);

public static final String HFILE_FILE_MAX_BYTES = "hoodie.hfile.max.file.size";
public static final String HFILE_BLOCK_SIZE_BYTES = "hoodie.hfile.block.size";
public static final String DEFAULT_HFILE_BLOCK_SIZE_BYTES = String.valueOf(1 * 1024 * 1024);
public static final String DEFAULT_HFILE_FILE_MAX_BYTES = String.valueOf(120 * 1024 * 1024);

public static final String ORC_FILE_MAX_BYTES = "hoodie.orc.max.file.size";
public static final String DEFAULT_ORC_FILE_MAX_BYTES = String.valueOf(120 * 1024 * 1024);
// size of the memory buffer in bytes for writing
public static final String ORC_STRIPE_SIZE = "hoodie.orc.stripe.size";
Copy link
Contributor

Choose a reason for hiding this comment

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

Can you please add comments on what is the stripe size used for ?

public static final String DEFAULT_ORC_STRIPE_SIZE = String.valueOf(64 * 1024 * 1024);
// file system block size
public static final String ORC_BLOCK_SIZE = "hoodie.orc.block.size";
Copy link
Contributor

Choose a reason for hiding this comment

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

Same for block size

public static final String DEFAULT_ORC_BLOCK_SIZE = DEFAULT_ORC_FILE_MAX_BYTES;

// used to size log files
public static final String LOGFILE_SIZE_MAX_BYTES = "hoodie.logfile.max.size";
public static final String DEFAULT_LOGFILE_SIZE_MAX_BYTES = String.valueOf(1024 * 1024 * 1024); // 1 GB
Expand All @@ -54,9 +65,11 @@ public class HoodieStorageConfig extends DefaultHoodieConfig {
public static final String DEFAULT_STREAM_COMPRESSION_RATIO = String.valueOf(0.1);
public static final String PARQUET_COMPRESSION_CODEC = "hoodie.parquet.compression.codec";
public static final String HFILE_COMPRESSION_ALGORITHM = "hoodie.hfile.compression.algorithm";
public static final String ORC_COMPRESSION_CODEC = "hoodie.orc.compression.codec";
// Default compression codec for parquet
public static final String DEFAULT_PARQUET_COMPRESSION_CODEC = "gzip";
public static final String DEFAULT_HFILE_COMPRESSION_ALGORITHM = "GZ";
public static final String DEFAULT_ORC_COMPRESSION_CODEC = "ZLIB";
public static final String LOGFILE_TO_PARQUET_COMPRESSION_RATIO = "hoodie.logfile.to.parquet.compression.ratio";
// Default compression ratio for log file to parquet, general 3x
public static final String DEFAULT_LOGFILE_TO_PARQUET_COMPRESSION_RATIO = String.valueOf(0.35);
Expand Down Expand Up @@ -140,6 +153,26 @@ public Builder logFileToParquetCompressionRatio(double logFileToParquetCompressi
return this;
}

public Builder orcMaxFileSize(long maxFileSize) {
props.setProperty(ORC_FILE_MAX_BYTES, String.valueOf(maxFileSize));
return this;
}

public Builder orcStripeSize(int orcStripeSize) {
props.setProperty(ORC_STRIPE_SIZE, String.valueOf(orcStripeSize));
return this;
}

public Builder orcBlockSize(int orcBlockSize) {
props.setProperty(ORC_BLOCK_SIZE, String.valueOf(orcBlockSize));
return this;
}

public Builder orcCompressionCodec(String orcCompressionCodec) {
props.setProperty(ORC_COMPRESSION_CODEC, orcCompressionCodec);
return this;
}

public HoodieStorageConfig build() {
HoodieStorageConfig config = new HoodieStorageConfig(props);
setDefaultOnCondition(props, !props.containsKey(PARQUET_FILE_MAX_BYTES), PARQUET_FILE_MAX_BYTES,
Expand All @@ -166,6 +199,15 @@ public HoodieStorageConfig build() {
setDefaultOnCondition(props, !props.containsKey(HFILE_FILE_MAX_BYTES), HFILE_FILE_MAX_BYTES,
DEFAULT_HFILE_FILE_MAX_BYTES);

setDefaultOnCondition(props, !props.containsKey(ORC_FILE_MAX_BYTES), ORC_FILE_MAX_BYTES,
DEFAULT_ORC_FILE_MAX_BYTES);
setDefaultOnCondition(props, !props.containsKey(ORC_STRIPE_SIZE), ORC_STRIPE_SIZE,
DEFAULT_ORC_STRIPE_SIZE);
setDefaultOnCondition(props, !props.containsKey(ORC_BLOCK_SIZE), ORC_BLOCK_SIZE,
DEFAULT_ORC_BLOCK_SIZE);
setDefaultOnCondition(props, !props.containsKey(ORC_COMPRESSION_CODEC), ORC_COMPRESSION_CODEC,
DEFAULT_ORC_COMPRESSION_CODEC);

return config;
}
}
Expand Down
Expand Up @@ -42,6 +42,7 @@
import org.apache.hudi.metrics.datadog.DatadogHttpClient.ApiSite;
import org.apache.hudi.table.action.compact.CompactionTriggerStrategy;
import org.apache.hudi.table.action.compact.strategy.CompactionStrategy;
import org.apache.orc.CompressionKind;
import org.apache.parquet.hadoop.metadata.CompressionCodecName;

import javax.annotation.concurrent.Immutable;
Expand Down Expand Up @@ -752,6 +753,22 @@ public Compression.Algorithm getHFileCompressionAlgorithm() {
return Compression.Algorithm.valueOf(props.getProperty(HoodieStorageConfig.HFILE_COMPRESSION_ALGORITHM));
}

public long getOrcMaxFileSize() {
return Long.parseLong(props.getProperty(HoodieStorageConfig.ORC_FILE_MAX_BYTES));
}

public int getOrcStripeSize() {
return Integer.parseInt(props.getProperty(HoodieStorageConfig.ORC_STRIPE_SIZE));
}

public int getOrcBlockSize() {
return Integer.parseInt(props.getProperty(HoodieStorageConfig.ORC_BLOCK_SIZE));
}

public CompressionKind getOrcCompressionCodec() {
return CompressionKind.valueOf(props.getProperty(HoodieStorageConfig.ORC_COMPRESSION_CODEC));
}

/**
* metrics properties.
*/
Expand Down
Expand Up @@ -18,6 +18,9 @@

package org.apache.hudi.io.storage;

import java.util.concurrent.atomic.AtomicLong;
import org.apache.avro.generic.GenericRecord;
import org.apache.hudi.avro.HoodieAvroUtils;
import org.apache.hudi.common.model.HoodieRecord;

import org.apache.avro.generic.IndexedRecord;
Expand All @@ -35,4 +38,11 @@ public interface HoodieFileWriter<R extends IndexedRecord> {
void writeAvro(String key, R oldRecord) throws IOException;

long getBytesWritten();

default void prepRecordWithMetadata(R avroRecord, HoodieRecord record, String instantTime, Integer partitionId, AtomicLong recordIndex, String fileName) {
String seqId = HoodieRecord.generateSequenceId(instantTime, partitionId, recordIndex.getAndIncrement());
HoodieAvroUtils.addHoodieKeyToRecord((GenericRecord) avroRecord, record.getRecordKey(), record.getPartitionPath(), fileName);
HoodieAvroUtils.addCommitMetadataToRecord((GenericRecord) avroRecord, instantTime, seqId);
return;
}
}
Expand Up @@ -34,6 +34,7 @@

import java.io.IOException;

import static org.apache.hudi.common.model.HoodieFileFormat.ORC;
import static org.apache.hudi.common.model.HoodieFileFormat.PARQUET;
import static org.apache.hudi.common.model.HoodieFileFormat.HFILE;

Expand All @@ -49,6 +50,9 @@ public static <T extends HoodieRecordPayload, R extends IndexedRecord, I, K, O>
if (HFILE.getFileExtension().equals(extension)) {
return newHFileFileWriter(instantTime, path, config, schema, hoodieTable, taskContextSupplier);
}
if (ORC.getFileExtension().equals(extension)) {
return newOrcFileWriter(instantTime, path, config, schema, hoodieTable, taskContextSupplier);
}
throw new UnsupportedOperationException(extension + " format not supported yet.");
}

Expand Down Expand Up @@ -77,6 +81,15 @@ private static <T extends HoodieRecordPayload, R extends IndexedRecord> HoodieFi
return new HoodieHFileWriter<>(instantTime, path, hfileConfig, schema, taskContextSupplier);
}

private static <T extends HoodieRecordPayload, R extends IndexedRecord> HoodieFileWriter<R> newOrcFileWriter(
String instantTime, Path path, HoodieWriteConfig config, Schema schema, HoodieTable hoodieTable,
TaskContextSupplier taskContextSupplier) throws IOException {
BloomFilter filter = createBloomFilter(config);
HoodieOrcConfig orcConfig = new HoodieOrcConfig(hoodieTable.getHadoopConf(), config.getOrcCompressionCodec(),
config.getOrcStripeSize(), config.getOrcBlockSize(), config.getOrcMaxFileSize(), filter);
return new HoodieOrcWriter<>(instantTime, path, orcConfig, schema, taskContextSupplier);
}

private static BloomFilter createBloomFilter(HoodieWriteConfig config) {
return BloomFilterFactory.createBloomFilter(config.getBloomFilterNumEntries(), config.getBloomFilterFPP(),
config.getDynamicBloomFilterMaxNumEntries(),
Expand Down
Expand Up @@ -99,13 +99,9 @@ public HoodieHFileWriter(String instantTime, Path file, HoodieHFileConfig hfileC

@Override
public void writeAvroWithMetadata(R avroRecord, HoodieRecord record) throws IOException {
String seqId =
HoodieRecord.generateSequenceId(instantTime, taskContextSupplier.getPartitionIdSupplier().get(), recordIndex.getAndIncrement());
HoodieAvroUtils.addHoodieKeyToRecord((GenericRecord) avroRecord, record.getRecordKey(), record.getPartitionPath(),
file.getName());
HoodieAvroUtils.addCommitMetadataToRecord((GenericRecord) avroRecord, instantTime, seqId);

writeAvro(record.getRecordKey(), (IndexedRecord)avroRecord);
prepRecordWithMetadata(avroRecord, record, instantTime,
taskContextSupplier.getPartitionIdSupplier().get(), recordIndex, file.getName());
writeAvro(record.getRecordKey(), (IndexedRecord) avroRecord);
}

@Override
Expand Down
@@ -0,0 +1,72 @@
/*
* 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.hudi.common.bloom.BloomFilter;
import org.apache.orc.CompressionKind;

public class HoodieOrcConfig {
static final String AVRO_SCHEMA_METADATA_KEY = "orc.avro.schema";

private final CompressionKind compressionKind;
private final int stripeSize;
private final int blockSize;
private final long maxFileSize;
private final Configuration hadoopConf;
private final BloomFilter bloomFilter;

public HoodieOrcConfig(Configuration hadoopConf, CompressionKind compressionKind, int stripeSize,
int blockSize, long maxFileSize, BloomFilter bloomFilter) {
this.hadoopConf = hadoopConf;
this.compressionKind = compressionKind;
this.stripeSize = stripeSize;
this.blockSize = blockSize;
this.maxFileSize = maxFileSize;
this.bloomFilter = bloomFilter;
}

public Configuration getHadoopConf() {
return hadoopConf;
}

public CompressionKind getCompressionKind() {
return compressionKind;
}

public int getStripeSize() {
return stripeSize;
}

public int getBlockSize() {
return blockSize;
}

public long getMaxFileSize() {
return maxFileSize;
}

public boolean useBloomFilter() {
return bloomFilter != null;
}

public BloomFilter getBloomFilter() {
return bloomFilter;
}
}