vinothchandar commented on a change in pull request #3889:
URL: https://github.com/apache/hudi/pull/3889#discussion_r755096705
##########
File path:
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieFileWriterFactory.java
##########
@@ -81,8 +84,13 @@
TaskContextSupplier taskContextSupplier) throws IOException {
BloomFilter filter = createBloomFilter(config);
- HoodieHFileConfig hfileConfig = new
HoodieHFileConfig(hoodieTable.getHadoopConf(),
- config.getHFileCompressionAlgorithm(), config.getHFileBlockSize(),
config.getHFileMaxFileSize(), filter);
+ HoodieHFileConfig hfileConfig;
+ if (!StringUtils.isNullOrEmpty(config.getHFileComparatorClassName())) {
Review comment:
can we have add a . `HoodieCommonKVComparator` and set this to be the
default. that way you ll always have a comparator here. This is a third
comparator, which can be common, in case we need it going forward.
##########
File path:
hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHBaseComparators.java
##########
@@ -0,0 +1,32 @@
+/*
+ * 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 HoodieHBaseComparators {
+ public static class HoodieHBaseKVComparator
Review comment:
why nest this? just a simple top level class?
##########
File path:
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileConfig.java
##########
@@ -34,19 +40,23 @@
private boolean dropBehindCacheCompaction;
private Configuration hadoopConf;
private BloomFilter bloomFilter;
-
- // This is private in CacheConfig so have been copied here.
- private static boolean DROP_BEHIND_CACHE_COMPACTION_DEFAULT = true;
+ private KeyValue.KVComparator hfileComparator;
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);
+ HColumnDescriptor.DEFAULT_CACHE_DATA_IN_L1,
DROP_BEHIND_CACHE_COMPACTION_DEFAULT, bloomFilter, defaultHFileComparator);
+ }
+
+ public HoodieHFileConfig(Configuration hadoopConf, Compression.Algorithm
compressionAlgorithm, int blockSize,
Review comment:
have 1 constructor that always takes a comparator?
##########
File path:
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieStorageConfig.java
##########
@@ -83,6 +83,14 @@
.withDocumentation("Lower values increase the size of metadata tracked
within HFile, but can offer potentially "
+ "faster lookup times.");
+ public static final ConfigProperty<String> HFILE_COMPARATOR_CLASS_NAME =
ConfigProperty
Review comment:
I am not sure we need to expose a user config here. All HFile usages are
more internal.
##########
File path:
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieFileWriterFactory.java
##########
@@ -81,8 +84,13 @@
TaskContextSupplier taskContextSupplier) throws IOException {
BloomFilter filter = createBloomFilter(config);
- HoodieHFileConfig hfileConfig = new
HoodieHFileConfig(hoodieTable.getHadoopConf(),
- config.getHFileCompressionAlgorithm(), config.getHFileBlockSize(),
config.getHFileMaxFileSize(), filter);
+ HoodieHFileConfig hfileConfig;
+ if (!StringUtils.isNullOrEmpty(config.getHFileComparatorClassName())) {
Review comment:
I guess `HoodieHBaseComparators.HoodieHBaseKVComparator()` is that? can
we just pass it in from here?
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]