diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieFileWriterFactory.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieFileWriterFactory.java index e88c34f60..c6d4540f2 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieFileWriterFactory.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieFileWriterFactory.java @@ -38,6 +38,10 @@ import java.io.IOException; import static org.apache.hudi.common.model.HoodieFileFormat.HFILE; import static org.apache.hudi.common.model.HoodieFileFormat.ORC; import static org.apache.hudi.common.model.HoodieFileFormat.PARQUET; +import static org.apache.hudi.io.storage.HoodieHFileConfig.CACHE_DATA_IN_L1; +import static org.apache.hudi.io.storage.HoodieHFileConfig.DROP_BEHIND_CACHE_COMPACTION; +import static org.apache.hudi.io.storage.HoodieHFileConfig.HFILE_COMPARATOR; +import static org.apache.hudi.io.storage.HoodieHFileConfig.PREFETCH_ON_OPEN; public class HoodieFileWriterFactory { @@ -82,7 +86,8 @@ public class HoodieFileWriterFactory { BloomFilter filter = createBloomFilter(config); HoodieHFileConfig hfileConfig = new HoodieHFileConfig(hoodieTable.getHadoopConf(), - config.getHFileCompressionAlgorithm(), config.getHFileBlockSize(), config.getHFileMaxFileSize(), filter); + config.getHFileCompressionAlgorithm(), config.getHFileBlockSize(), config.getHFileMaxFileSize(), + PREFETCH_ON_OPEN, CACHE_DATA_IN_L1, DROP_BEHIND_CACHE_COMPACTION, filter, HFILE_COMPARATOR); return new HoodieHFileWriter<>(instantTime, path, hfileConfig, schema, taskContextSupplier); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileConfig.java index 031f92ccc..7e4c519a8 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileConfig.java @@ -18,35 +18,35 @@ package org.apache.hudi.io.storage; +import org.apache.hudi.common.bloom.BloomFilter; + import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HColumnDescriptor; +import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.io.compress.Compression; import org.apache.hadoop.hbase.io.hfile.CacheConfig; -import org.apache.hudi.common.bloom.BloomFilter; public class HoodieHFileConfig { - private Compression.Algorithm compressionAlgorithm; - private int blockSize; - private long maxFileSize; - private boolean prefetchBlocksOnOpen; - private boolean cacheDataInL1; - private boolean dropBehindCacheCompaction; - private Configuration hadoopConf; - private BloomFilter bloomFilter; - + public static final KeyValue.KVComparator HFILE_COMPARATOR = new HoodieHBaseKVComparator(); + public static final boolean PREFETCH_ON_OPEN = CacheConfig.DEFAULT_PREFETCH_ON_OPEN; + public static final boolean CACHE_DATA_IN_L1 = HColumnDescriptor.DEFAULT_CACHE_DATA_IN_L1; // This is private in CacheConfig so have been copied here. - private static boolean DROP_BEHIND_CACHE_COMPACTION_DEFAULT = true; + public static final boolean DROP_BEHIND_CACHE_COMPACTION = true; - public HoodieHFileConfig(Configuration hadoopConf, Compression.Algorithm compressionAlgorithm, int blockSize, - long maxFileSize, BloomFilter bloomFilter) { - this(hadoopConf, compressionAlgorithm, blockSize, maxFileSize, CacheConfig.DEFAULT_PREFETCH_ON_OPEN, - HColumnDescriptor.DEFAULT_CACHE_DATA_IN_L1, DROP_BEHIND_CACHE_COMPACTION_DEFAULT, bloomFilter); - } + private final Compression.Algorithm compressionAlgorithm; + private final int blockSize; + private final long maxFileSize; + private final boolean prefetchBlocksOnOpen; + private final boolean cacheDataInL1; + private final boolean dropBehindCacheCompaction; + private final Configuration hadoopConf; + private final BloomFilter bloomFilter; + private final KeyValue.KVComparator hfileComparator; public HoodieHFileConfig(Configuration hadoopConf, Compression.Algorithm compressionAlgorithm, int blockSize, long maxFileSize, boolean prefetchBlocksOnOpen, boolean cacheDataInL1, - boolean dropBehindCacheCompaction, BloomFilter bloomFilter) { + boolean dropBehindCacheCompaction, BloomFilter bloomFilter, KeyValue.KVComparator hfileComparator) { this.hadoopConf = hadoopConf; this.compressionAlgorithm = compressionAlgorithm; this.blockSize = blockSize; @@ -55,6 +55,7 @@ public class HoodieHFileConfig { this.cacheDataInL1 = cacheDataInL1; this.dropBehindCacheCompaction = dropBehindCacheCompaction; this.bloomFilter = bloomFilter; + this.hfileComparator = hfileComparator; } public Configuration getHadoopConf() { @@ -92,4 +93,8 @@ public class HoodieHFileConfig { public BloomFilter getBloomFilter() { return bloomFilter; } + + public KeyValue.KVComparator getHfileComparator() { + return hfileComparator; + } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileWriter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileWriter.java index 6747c4a4a..d7e01c50c 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileWriter.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileWriter.java @@ -25,6 +25,7 @@ import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.fs.HoodieWrapperFileSystem; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; + import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.IndexedRecord; @@ -85,14 +86,18 @@ public class HoodieHFileWriter sortedRecordsMap = new TreeMap<>(); diff --git a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHBaseKVComparator.java b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHBaseKVComparator.java new file mode 100644 index 000000000..2d4d96959 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHBaseKVComparator.java @@ -0,0 +1,29 @@ +/* + * 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.hbase.KeyValue; + +/** + * This class is explicitly used as Key Comparator to work around the hard coded + * legacy format class names inside HBase. Otherwise, we will face issues with shading. + */ +public class HoodieHBaseKVComparator extends KeyValue.KVComparator { +} diff --git a/hudi-common/src/test/java/org/apache/hudi/common/fs/inline/TestInLineFileSystemHFileInLining.java b/hudi-common/src/test/java/org/apache/hudi/common/fs/inline/TestInLineFileSystemHFileInLining.java index 4122d500b..cc59b4602 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/fs/inline/TestInLineFileSystemHFileInLining.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/fs/inline/TestInLineFileSystemHFileInLining.java @@ -19,6 +19,7 @@ package org.apache.hudi.common.fs.inline; import org.apache.hudi.common.testutils.FileSystemTestUtils; +import org.apache.hudi.io.storage.HoodieHBaseKVComparator; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataInputStream; @@ -92,7 +93,7 @@ public class TestInLineFileSystemHFileInLining { HFile.Writer writer = HFile.getWriterFactory(inMemoryConf, cacheConf) .withOutputStream(fout) .withFileContext(meta) - .withComparator(new KeyValue.KVComparator()) + .withComparator(new HoodieHBaseKVComparator()) .create(); writeRecords(writer);