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-960] Implementation of the HFile base and log file format. #1804

Merged
merged 4 commits into from Aug 31, 2020
Merged
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
6 changes: 6 additions & 0 deletions hudi-client/pom.xml
Expand Up @@ -206,6 +206,12 @@
<artifactId>hbase-client</artifactId>
<version>${hbase.version}</version>
</dependency>
<dependency>
<groupId>org.apache.hbase</groupId>
<artifactId>hbase-server</artifactId>
<version>${hbase.version}</version>
<scope>provided</scope>
</dependency>

<!-- Hoodie - Tests -->
<dependency>
Expand Down
Expand Up @@ -39,6 +39,10 @@ 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);
// 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 @@ -49,8 +53,10 @@ public class HoodieStorageConfig extends DefaultHoodieConfig {
// Default compression ratio for parquet
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";
// 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 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 @@ -79,7 +85,7 @@ public Builder fromProperties(Properties props) {
return this;
}

public Builder limitFileSize(long maxFileSize) {
public Builder parquetMaxFileSize(long maxFileSize) {
props.setProperty(PARQUET_FILE_MAX_BYTES, String.valueOf(maxFileSize));
return this;
}
Expand All @@ -94,6 +100,16 @@ public Builder parquetPageSize(int pageSize) {
return this;
}

public Builder hfileMaxFileSize(long maxFileSize) {
props.setProperty(HFILE_FILE_MAX_BYTES, String.valueOf(maxFileSize));
Copy link
Contributor

Choose a reason for hiding this comment

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

@vinothchandar : wrt you comment on having two diff configs. I see similar configs at other places too. like bloom index parallelism, we have one config per index type. Initially I thought we will have any one config which will be used by any index type that is being initialized. But I saw that every index has its own set of configs and don't share any.

Copy link
Member

Choose a reason for hiding this comment

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

not following. sorry. are you suggesting having a single config or two?
So, we need to have a config per usage of HFile. so we can control the base file size for data, metadata, record index separately.

We cannot have a generic base.file.size or hfile.size config here, at this level IMO. cc @prashantwason

Copy link
Member

Choose a reason for hiding this comment

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

I see that you split this function into file specific functions. That doable but with more base file formats added, it may be cumbersome and verbose to keep adding the .limitXXXFileSize for specific formats.

@prashantwason I think we need to eventually have a config "per use" of base file - data, metadata, index - since people may want to control them differently. So, in that sense, this has to kind of change.
yes the change is backwards compatible to RDD clients (which I thought was okay, since its just uber. if you prefer to not have that, lmk. IMO, its about time, we cleaned these up, given we are moving to having way more base files/tables in the mix)

return this;
}

public Builder hfileBlockSize(int blockSize) {
props.setProperty(HFILE_BLOCK_SIZE_BYTES, String.valueOf(blockSize));
return this;
}

public Builder logFileDataBlockMaxSize(int dataBlockSize) {
props.setProperty(LOGFILE_DATA_BLOCK_SIZE_MAX_BYTES, String.valueOf(dataBlockSize));
return this;
Expand All @@ -114,6 +130,11 @@ public Builder parquetCompressionCodec(String parquetCompressionCodec) {
return this;
}

public Builder hfileCompressionAlgorithm(String hfileCompressionAlgorithm) {
props.setProperty(HFILE_COMPRESSION_ALGORITHM, hfileCompressionAlgorithm);
return this;
}

public Builder logFileToParquetCompressionRatio(double logFileToParquetCompressionRatio) {
props.setProperty(LOGFILE_TO_PARQUET_COMPRESSION_RATIO, String.valueOf(logFileToParquetCompressionRatio));
return this;
Expand All @@ -137,6 +158,14 @@ public HoodieStorageConfig build() {
DEFAULT_PARQUET_COMPRESSION_CODEC);
setDefaultOnCondition(props, !props.containsKey(LOGFILE_TO_PARQUET_COMPRESSION_RATIO),
LOGFILE_TO_PARQUET_COMPRESSION_RATIO, DEFAULT_LOGFILE_TO_PARQUET_COMPRESSION_RATIO);

setDefaultOnCondition(props, !props.containsKey(HFILE_BLOCK_SIZE_BYTES), HFILE_BLOCK_SIZE_BYTES,
DEFAULT_HFILE_BLOCK_SIZE_BYTES);
setDefaultOnCondition(props, !props.containsKey(HFILE_COMPRESSION_ALGORITHM), HFILE_COMPRESSION_ALGORITHM,
DEFAULT_HFILE_COMPRESSION_ALGORITHM);
setDefaultOnCondition(props, !props.containsKey(HFILE_FILE_MAX_BYTES), HFILE_FILE_MAX_BYTES,
DEFAULT_HFILE_FILE_MAX_BYTES);

return config;
}
}
Expand Down
Expand Up @@ -18,6 +18,7 @@

package org.apache.hudi.config;

import org.apache.hadoop.hbase.io.compress.Compression;
import org.apache.hudi.client.HoodieWriteClient;
import org.apache.hudi.client.WriteStatus;
import org.apache.hudi.client.bootstrap.BootstrapMode;
Expand Down Expand Up @@ -55,6 +56,8 @@
@Immutable
public class HoodieWriteConfig extends DefaultHoodieConfig {

private static final long serialVersionUID = 0L;

public static final String TABLE_NAME = "hoodie.table.name";
public static final String DEFAULT_ROLLBACK_USING_MARKERS = "false";
public static final String ROLLBACK_USING_MARKERS = "hoodie.rollback.using.markers";
Expand Down Expand Up @@ -556,6 +559,18 @@ public double getLogFileToParquetCompressionRatio() {
return Double.parseDouble(props.getProperty(HoodieStorageConfig.LOGFILE_TO_PARQUET_COMPRESSION_RATIO));
}

public long getHFileMaxFileSize() {
return Long.parseLong(props.getProperty(HoodieStorageConfig.HFILE_FILE_MAX_BYTES));
}

public int getHFileBlockSize() {
return Integer.parseInt(props.getProperty(HoodieStorageConfig.HFILE_BLOCK_SIZE_BYTES));
}

public Compression.Algorithm getHFileCompressionAlgorithm() {
return Compression.Algorithm.valueOf(props.getProperty(HoodieStorageConfig.HFILE_COMPRESSION_ALGORITHM));
}

/**
* metrics properties.
*/
Expand Down
Expand Up @@ -45,6 +45,7 @@

import java.io.IOException;
import java.util.Iterator;
import java.util.Map;

public class HoodieCreateHandle<T extends HoodieRecordPayload> extends HoodieWriteHandle<T> {

Expand All @@ -55,7 +56,7 @@ public class HoodieCreateHandle<T extends HoodieRecordPayload> extends HoodieWri
private long recordsWritten = 0;
private long insertRecordsWritten = 0;
private long recordsDeleted = 0;
private Iterator<HoodieRecord<T>> recordIterator;
private Map<String, HoodieRecord<T>> recordMap;
vinothchandar marked this conversation as resolved.
Show resolved Hide resolved
private boolean useWriterSchema = false;

public HoodieCreateHandle(HoodieWriteConfig config, String instantTime, HoodieTable<T> hoodieTable,
Expand Down Expand Up @@ -90,9 +91,10 @@ public HoodieCreateHandle(HoodieWriteConfig config, String instantTime, HoodieTa
* Called by the compactor code path.
*/
public HoodieCreateHandle(HoodieWriteConfig config, String instantTime, HoodieTable<T> hoodieTable,
String partitionPath, String fileId, Iterator<HoodieRecord<T>> recordIterator, SparkTaskContextSupplier sparkTaskContextSupplier) {
String partitionPath, String fileId, Map<String, HoodieRecord<T>> recordMap,
Copy link
Contributor

Choose a reason for hiding this comment

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

Not sure if we can leak the type of base file to compactor. But did you think about having two overloaded methods here. So for parquet compaction path, iterator will be passed in, where as for hfile compaction, record map will be passed in.

Copy link
Member

Choose a reason for hiding this comment

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

Ideally not. the more the compactor can function wihtout knowing the base file type specifics, the better

SparkTaskContextSupplier sparkTaskContextSupplier) {
this(config, instantTime, hoodieTable, partitionPath, fileId, sparkTaskContextSupplier);
this.recordIterator = recordIterator;
this.recordMap = recordMap;
this.useWriterSchema = true;
}

Expand Down Expand Up @@ -138,9 +140,17 @@ public void write(HoodieRecord record, Option<IndexedRecord> avroRecord) {
* Writes all records passed.
*/
public void write() {
Iterator<String> keyIterator;
if (hoodieTable.requireSortedRecords()) {
// Sorting the keys limits the amount of extra memory required for writing sorted records
keyIterator = recordMap.keySet().stream().sorted().iterator();
} else {
keyIterator = recordMap.keySet().stream().iterator();
}
try {
while (recordIterator.hasNext()) {
HoodieRecord<T> record = recordIterator.next();
while (keyIterator.hasNext()) {
final String key = keyIterator.next();
HoodieRecord<T> record = recordMap.get(key);
if (useWriterSchema) {
write(record, record.getData().getInsertValue(writerSchemaWithMetafields));
} else {
Expand Down
Expand Up @@ -58,16 +58,17 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieWrit

private static final Logger LOG = LogManager.getLogger(HoodieMergeHandle.class);

private Map<String, HoodieRecord<T>> keyToNewRecords;
private Set<String> writtenRecordKeys;
protected Map<String, HoodieRecord<T>> keyToNewRecords;
protected Set<String> writtenRecordKeys;
private HoodieFileWriter<IndexedRecord> fileWriter;

private Path newFilePath;
private Path oldFilePath;
private long recordsWritten = 0;
private long recordsDeleted = 0;
private long updatedRecordsWritten = 0;
private long insertRecordsWritten = 0;
private boolean useWriterSchema;
protected long insertRecordsWritten = 0;
protected boolean useWriterSchema;
private HoodieBaseFile baseFileToMerge;

public HoodieMergeHandle(HoodieWriteConfig config, String instantTime, HoodieTable<T> hoodieTable,
Expand Down Expand Up @@ -179,7 +180,7 @@ private boolean writeUpdateRecord(HoodieRecord<T> hoodieRecord, Option<IndexedRe
return writeRecord(hoodieRecord, indexedRecord);
}

private boolean writeRecord(HoodieRecord<T> hoodieRecord, Option<IndexedRecord> indexedRecord) {
protected boolean writeRecord(HoodieRecord<T> hoodieRecord, Option<IndexedRecord> indexedRecord) {
Option recordMetadata = hoodieRecord.getData().getMetadata();
if (!partitionPath.equals(hoodieRecord.getPartitionPath())) {
HoodieUpsertException failureEx = new HoodieUpsertException("mismatched partition path, record partition: "
Expand Down
@@ -0,0 +1,126 @@
/*
* 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;

import org.apache.hudi.client.SparkTaskContextSupplier;
import org.apache.hudi.client.WriteStatus;
import org.apache.hudi.common.model.HoodieBaseFile;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieUpsertException;
import org.apache.hudi.table.HoodieTable;

import org.apache.avro.generic.GenericRecord;

import java.io.IOException;
import java.util.Iterator;
import java.util.Map;
import java.util.PriorityQueue;
import java.util.Queue;

/**
* Hoodie merge handle which writes records (new inserts or updates) sorted by their key.
*
* The implementation performs a merge-sort by comparing the key of the record being written to the list of
* keys in newRecordKeys (sorted in-memory).
*/
public class HoodieSortedMergeHandle<T extends HoodieRecordPayload> extends HoodieMergeHandle<T> {

private Queue<String> newRecordKeysSorted = new PriorityQueue<>();
vinothchandar marked this conversation as resolved.
Show resolved Hide resolved

public HoodieSortedMergeHandle(HoodieWriteConfig config, String instantTime, HoodieTable<T> hoodieTable,
Iterator<HoodieRecord<T>> recordItr, String partitionPath, String fileId, SparkTaskContextSupplier sparkTaskContextSupplier) {
super(config, instantTime, hoodieTable, recordItr, partitionPath, fileId, sparkTaskContextSupplier);
newRecordKeysSorted.addAll(keyToNewRecords.keySet());
}

/**
* Called by compactor code path.
*/
public HoodieSortedMergeHandle(HoodieWriteConfig config, String instantTime, HoodieTable<T> hoodieTable,
vinothchandar marked this conversation as resolved.
Show resolved Hide resolved
Map<String, HoodieRecord<T>> keyToNewRecordsOrig, String partitionPath, String fileId,
HoodieBaseFile dataFileToBeMerged, SparkTaskContextSupplier sparkTaskContextSupplier) {
super(config, instantTime, hoodieTable, keyToNewRecordsOrig, partitionPath, fileId, dataFileToBeMerged,
sparkTaskContextSupplier);

newRecordKeysSorted.addAll(keyToNewRecords.keySet());
}

/**
* Go through an old record. Here if we detect a newer version shows up, we write the new one to the file.
*/
@Override
public void write(GenericRecord oldRecord) {
String key = oldRecord.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString();

// To maintain overall sorted order across updates and inserts, write any new inserts whose keys are less than
// the oldRecord's key.
while (!newRecordKeysSorted.isEmpty() && newRecordKeysSorted.peek().compareTo(key) <= 0) {
String keyToPreWrite = newRecordKeysSorted.remove();
Copy link
Member

Choose a reason for hiding this comment

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

instead, we can just do a streaming sort-merge?

Copy link
Member

Choose a reason for hiding this comment

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

cc @prashantwason can you please chime in. I feel we can avoid the queue altogether and just sort merge?

Copy link
Member Author

Choose a reason for hiding this comment

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

This is a streaming sort-merge. The logic is as follows:

  1. Hold the keys of records which have changed in memory (in a PriorityQueue). This is in addition to the Map whic his already there in HoodieMergeHandle.
  2. For each write()
    • if the key of the record being written < head of PriorityQueue => Write out this record
    • if the key of the record being written > head of PriorityQueue => Write out all records with smaller keys

Do you have some other algorithm in mind?

Copy link
Member

Choose a reason for hiding this comment

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

I am thinking we don't need the map in HoodieMergeHandle or the priorityQueue. The record which have changeed i.e. the input iterator is already sorted. lets call it inputItr

So , we can just compare the recordBeingWritten with inputItr.next() and write out the smallest one, if equal, we call the payload to merge.

This will avoid any kind of memory overhead

Copy link
Member Author

Choose a reason for hiding this comment

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

If the inputItr is sorted then yes all this overhead can be removed.

if (keyToPreWrite.equals(key)) {
// will be handled as an update later
break;
}

// This is a new insert
HoodieRecord<T> hoodieRecord = new HoodieRecord<>(keyToNewRecords.get(keyToPreWrite));
if (writtenRecordKeys.contains(keyToPreWrite)) {
throw new HoodieUpsertException("Insert/Update not in sorted order");
}
try {
if (useWriterSchema) {
writeRecord(hoodieRecord, hoodieRecord.getData().getInsertValue(writerSchemaWithMetafields));
} else {
writeRecord(hoodieRecord, hoodieRecord.getData().getInsertValue(writerSchema));
}
insertRecordsWritten++;
writtenRecordKeys.add(keyToPreWrite);
} catch (IOException e) {
throw new HoodieUpsertException("Failed to write records", e);
}
}

super.write(oldRecord);
}

@Override
public WriteStatus close() {
// write out any pending records (this can happen when inserts are turned into updates)
newRecordKeysSorted.stream().forEach(key -> {
try {
HoodieRecord<T> hoodieRecord = keyToNewRecords.get(key);
if (!writtenRecordKeys.contains(hoodieRecord.getRecordKey())) {
if (useWriterSchema) {
writeRecord(hoodieRecord, hoodieRecord.getData().getInsertValue(writerSchemaWithMetafields));
} else {
writeRecord(hoodieRecord, hoodieRecord.getData().getInsertValue(writerSchema));
}
insertRecordsWritten++;
}
} catch (IOException e) {
throw new HoodieUpsertException("Failed to close UpdateHandle", e);
}
});
newRecordKeysSorted.clear();
keyToNewRecords.clear();

return super.close();
}
}
Expand Up @@ -35,6 +35,7 @@
import java.io.IOException;

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

public class HoodieFileWriterFactory {

Expand All @@ -45,16 +46,16 @@ public static <T extends HoodieRecordPayload, R extends IndexedRecord> HoodieFil
if (PARQUET.getFileExtension().equals(extension)) {
return newParquetFileWriter(instantTime, path, config, schema, hoodieTable, sparkTaskContextSupplier);
}
if (HFILE.getFileExtension().equals(extension)) {
return newHFileFileWriter(instantTime, path, config, schema, hoodieTable, sparkTaskContextSupplier);
}
throw new UnsupportedOperationException(extension + " format not supported yet.");
}

private static <T extends HoodieRecordPayload, R extends IndexedRecord> HoodieFileWriter<R> newParquetFileWriter(
String instantTime, Path path, HoodieWriteConfig config, Schema schema, HoodieTable hoodieTable,
SparkTaskContextSupplier sparkTaskContextSupplier) throws IOException {
BloomFilter filter = BloomFilterFactory
.createBloomFilter(config.getBloomFilterNumEntries(), config.getBloomFilterFPP(),
config.getDynamicBloomFilterMaxNumEntries(),
config.getBloomFilterType());
BloomFilter filter = createBloomFilter(config);
HoodieAvroWriteSupport writeSupport =
new HoodieAvroWriteSupport(new AvroSchemaConverter().convert(schema), schema, filter);

Expand All @@ -64,4 +65,21 @@ private static <T extends HoodieRecordPayload, R extends IndexedRecord> HoodieFi

return new HoodieParquetWriter<>(instantTime, path, parquetConfig, schema, sparkTaskContextSupplier);
}

private static <T extends HoodieRecordPayload, R extends IndexedRecord> HoodieFileWriter<R> newHFileFileWriter(
String instantTime, Path path, HoodieWriteConfig config, Schema schema, HoodieTable hoodieTable,
SparkTaskContextSupplier sparkTaskContextSupplier) throws IOException {

BloomFilter filter = createBloomFilter(config);
HoodieHFileConfig hfileConfig = new HoodieHFileConfig(hoodieTable.getHadoopConf(),
config.getHFileCompressionAlgorithm(), config.getHFileBlockSize(), config.getHFileMaxFileSize(), filter);

return new HoodieHFileWriter<>(instantTime, path, hfileConfig, schema, sparkTaskContextSupplier);
}

private static BloomFilter createBloomFilter(HoodieWriteConfig config) {
return BloomFilterFactory.createBloomFilter(config.getBloomFilterNumEntries(), config.getBloomFilterFPP(),
config.getDynamicBloomFilterMaxNumEntries(),
config.getBloomFilterType());
}
}