[HUDI-1180] Upgrade HBase to 2.4.9 (#5004)
Co-authored-by: Sagar Sumit <sagarsumit09@gmail.com>
This commit is contained in:
@@ -548,7 +548,8 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload, I, K, O> extends
|
||||
case AVRO_DATA_BLOCK:
|
||||
return new HoodieAvroDataBlock(recordList, header, keyField);
|
||||
case HFILE_DATA_BLOCK:
|
||||
return new HoodieHFileDataBlock(recordList, header, writeConfig.getHFileCompressionAlgorithm());
|
||||
return new HoodieHFileDataBlock(
|
||||
recordList, header, writeConfig.getHFileCompressionAlgorithm(), new Path(writeConfig.getBasePath()));
|
||||
case PARQUET_DATA_BLOCK:
|
||||
return new HoodieParquetDataBlock(recordList, header, keyField, writeConfig.getParquetCompressionCodec());
|
||||
default:
|
||||
|
||||
@@ -30,6 +30,7 @@ import org.apache.hudi.table.HoodieTable;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.parquet.avro.AvroSchemaConverter;
|
||||
|
||||
@@ -53,10 +54,12 @@ public class HoodieFileWriterFactory {
|
||||
return newParquetFileWriter(instantTime, path, config, schema, hoodieTable, taskContextSupplier, config.populateMetaFields());
|
||||
}
|
||||
if (HFILE.getFileExtension().equals(extension)) {
|
||||
return newHFileFileWriter(instantTime, path, config, schema, hoodieTable, taskContextSupplier);
|
||||
return newHFileFileWriter(
|
||||
instantTime, path, config, schema, hoodieTable.getHadoopConf(), taskContextSupplier);
|
||||
}
|
||||
if (ORC.getFileExtension().equals(extension)) {
|
||||
return newOrcFileWriter(instantTime, path, config, schema, hoodieTable, taskContextSupplier);
|
||||
return newOrcFileWriter(
|
||||
instantTime, path, config, schema, hoodieTable.getHadoopConf(), taskContextSupplier);
|
||||
}
|
||||
throw new UnsupportedOperationException(extension + " format not supported yet.");
|
||||
}
|
||||
@@ -64,28 +67,29 @@ public class HoodieFileWriterFactory {
|
||||
private static <T extends HoodieRecordPayload, R extends IndexedRecord> HoodieFileWriter<R> newParquetFileWriter(
|
||||
String instantTime, Path path, HoodieWriteConfig config, Schema schema, HoodieTable hoodieTable,
|
||||
TaskContextSupplier taskContextSupplier, boolean populateMetaFields) throws IOException {
|
||||
return newParquetFileWriter(instantTime, path, config, schema, hoodieTable, taskContextSupplier, populateMetaFields, populateMetaFields);
|
||||
return newParquetFileWriter(instantTime, path, config, schema, hoodieTable.getHadoopConf(),
|
||||
taskContextSupplier, populateMetaFields, populateMetaFields);
|
||||
}
|
||||
|
||||
private static <T extends HoodieRecordPayload, R extends IndexedRecord> HoodieFileWriter<R> newParquetFileWriter(
|
||||
String instantTime, Path path, HoodieWriteConfig config, Schema schema, HoodieTable hoodieTable,
|
||||
String instantTime, Path path, HoodieWriteConfig config, Schema schema, Configuration conf,
|
||||
TaskContextSupplier taskContextSupplier, boolean populateMetaFields, boolean enableBloomFilter) throws IOException {
|
||||
Option<BloomFilter> filter = enableBloomFilter ? Option.of(createBloomFilter(config)) : Option.empty();
|
||||
HoodieAvroWriteSupport writeSupport = new HoodieAvroWriteSupport(new AvroSchemaConverter(hoodieTable.getHadoopConf()).convert(schema), schema, filter);
|
||||
HoodieAvroWriteSupport writeSupport = new HoodieAvroWriteSupport(new AvroSchemaConverter(conf).convert(schema), schema, filter);
|
||||
|
||||
HoodieAvroParquetConfig parquetConfig = new HoodieAvroParquetConfig(writeSupport, config.getParquetCompressionCodec(),
|
||||
config.getParquetBlockSize(), config.getParquetPageSize(), config.getParquetMaxFileSize(),
|
||||
hoodieTable.getHadoopConf(), config.getParquetCompressionRatio(), config.parquetDictionaryEnabled());
|
||||
conf, config.getParquetCompressionRatio(), config.parquetDictionaryEnabled());
|
||||
|
||||
return new HoodieParquetWriter<>(instantTime, path, parquetConfig, schema, taskContextSupplier, populateMetaFields);
|
||||
}
|
||||
|
||||
private static <T extends HoodieRecordPayload, R extends IndexedRecord> HoodieFileWriter<R> newHFileFileWriter(
|
||||
String instantTime, Path path, HoodieWriteConfig config, Schema schema, HoodieTable hoodieTable,
|
||||
static <T extends HoodieRecordPayload, R extends IndexedRecord> HoodieFileWriter<R> newHFileFileWriter(
|
||||
String instantTime, Path path, HoodieWriteConfig config, Schema schema, Configuration conf,
|
||||
TaskContextSupplier taskContextSupplier) throws IOException {
|
||||
|
||||
BloomFilter filter = createBloomFilter(config);
|
||||
HoodieHFileConfig hfileConfig = new HoodieHFileConfig(hoodieTable.getHadoopConf(),
|
||||
HoodieHFileConfig hfileConfig = new HoodieHFileConfig(conf,
|
||||
config.getHFileCompressionAlgorithm(), config.getHFileBlockSize(), config.getHFileMaxFileSize(),
|
||||
HoodieHFileReader.KEY_FIELD_NAME, PREFETCH_ON_OPEN, CACHE_DATA_IN_L1, DROP_BEHIND_CACHE_COMPACTION,
|
||||
filter, HFILE_COMPARATOR);
|
||||
@@ -94,10 +98,10 @@ public class HoodieFileWriterFactory {
|
||||
}
|
||||
|
||||
private static <T extends HoodieRecordPayload, R extends IndexedRecord> HoodieFileWriter<R> newOrcFileWriter(
|
||||
String instantTime, Path path, HoodieWriteConfig config, Schema schema, HoodieTable hoodieTable,
|
||||
String instantTime, Path path, HoodieWriteConfig config, Schema schema, Configuration conf,
|
||||
TaskContextSupplier taskContextSupplier) throws IOException {
|
||||
BloomFilter filter = createBloomFilter(config);
|
||||
HoodieOrcConfig orcConfig = new HoodieOrcConfig(hoodieTable.getHadoopConf(), config.getOrcCompressionCodec(),
|
||||
HoodieOrcConfig orcConfig = new HoodieOrcConfig(conf, config.getOrcCompressionCodec(),
|
||||
config.getOrcStripeSize(), config.getOrcBlockSize(), config.getOrcMaxFileSize(), filter);
|
||||
return new HoodieOrcWriter<>(instantTime, path, orcConfig, schema, taskContextSupplier);
|
||||
}
|
||||
|
||||
@@ -21,14 +21,14 @@ package org.apache.hudi.io.storage;
|
||||
import org.apache.hudi.common.bloom.BloomFilter;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
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;
|
||||
|
||||
public class HoodieHFileConfig {
|
||||
|
||||
public static final KeyValue.KVComparator HFILE_COMPARATOR = new HoodieHBaseKVComparator();
|
||||
public static final CellComparator 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.
|
||||
@@ -42,12 +42,12 @@ public class HoodieHFileConfig {
|
||||
private final boolean dropBehindCacheCompaction;
|
||||
private final Configuration hadoopConf;
|
||||
private final BloomFilter bloomFilter;
|
||||
private final KeyValue.KVComparator hfileComparator;
|
||||
private final CellComparator hfileComparator;
|
||||
private final String keyFieldName;
|
||||
|
||||
public HoodieHFileConfig(Configuration hadoopConf, Compression.Algorithm compressionAlgorithm, int blockSize,
|
||||
long maxFileSize, String keyFieldName, boolean prefetchBlocksOnOpen, boolean cacheDataInL1,
|
||||
boolean dropBehindCacheCompaction, BloomFilter bloomFilter, KeyValue.KVComparator hfileComparator) {
|
||||
boolean dropBehindCacheCompaction, BloomFilter bloomFilter, CellComparator hfileComparator) {
|
||||
this.hadoopConf = hadoopConf;
|
||||
this.compressionAlgorithm = compressionAlgorithm;
|
||||
this.blockSize = blockSize;
|
||||
@@ -96,7 +96,7 @@ public class HoodieHFileConfig {
|
||||
return bloomFilter;
|
||||
}
|
||||
|
||||
public KeyValue.KVComparator getHfileComparator() {
|
||||
public CellComparator getHFileComparator() {
|
||||
return hfileComparator;
|
||||
}
|
||||
|
||||
|
||||
@@ -25,6 +25,8 @@ 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.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.StringUtils;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
@@ -38,8 +40,6 @@ import org.apache.hadoop.hbase.io.hfile.HFile;
|
||||
import org.apache.hadoop.hbase.io.hfile.HFileContext;
|
||||
import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
|
||||
import org.apache.hadoop.io.Writable;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.StringUtils;
|
||||
|
||||
import java.io.DataInput;
|
||||
import java.io.DataOutput;
|
||||
@@ -95,6 +95,7 @@ public class HoodieHFileWriter<T extends HoodieRecordPayload, R extends IndexedR
|
||||
|
||||
HFileContext context = new HFileContextBuilder().withBlockSize(hfileConfig.getBlockSize())
|
||||
.withCompression(hfileConfig.getCompressionAlgorithm())
|
||||
.withCellComparator(hfileConfig.getHFileComparator())
|
||||
.build();
|
||||
|
||||
conf.set(CacheConfig.PREFETCH_BLOCKS_ON_OPEN_KEY, String.valueOf(hfileConfig.shouldPrefetchBlocksOnOpen()));
|
||||
@@ -104,7 +105,6 @@ public class HoodieHFileWriter<T extends HoodieRecordPayload, R extends IndexedR
|
||||
this.writer = HFile.getWriterFactory(conf, cacheConfig)
|
||||
.withPath(this.fs, this.file)
|
||||
.withFileContext(context)
|
||||
.withComparator(hfileConfig.getHfileComparator())
|
||||
.create();
|
||||
|
||||
writer.appendFileInfo(HoodieHFileReader.KEY_SCHEMA.getBytes(), schema.toString().getBytes());
|
||||
|
||||
Reference in New Issue
Block a user