[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());
|
||||
|
||||
@@ -18,71 +18,113 @@
|
||||
|
||||
package org.apache.hudi.io.storage;
|
||||
|
||||
import org.apache.hudi.common.bloom.BloomFilter;
|
||||
import org.apache.hudi.common.bloom.BloomFilterFactory;
|
||||
import org.apache.hudi.common.bloom.BloomFilterTypeCode;
|
||||
import org.apache.hudi.common.bootstrap.index.HFileBootstrapIndex;
|
||||
import org.apache.hudi.common.engine.TaskContextSupplier;
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.common.model.EmptyHoodieRecordPayload;
|
||||
import org.apache.hudi.common.model.HoodieAvroRecord;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.util.FileIOUtils;
|
||||
import org.apache.hudi.config.HoodieIndexConfig;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericData;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.io.compress.Compression;
|
||||
import org.apache.hadoop.hbase.CellComparatorImpl;
|
||||
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
|
||||
import org.apache.hadoop.hbase.io.hfile.HFile;
|
||||
import org.apache.hadoop.hbase.util.Pair;
|
||||
import org.junit.jupiter.api.AfterEach;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.io.TempDir;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.params.ParameterizedTest;
|
||||
import org.junit.jupiter.params.provider.Arguments;
|
||||
import org.junit.jupiter.params.provider.MethodSource;
|
||||
import org.junit.jupiter.params.provider.ValueSource;
|
||||
import org.mockito.Mockito;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.function.Supplier;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.IntStream;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
import static org.apache.hudi.common.testutils.FileSystemTestUtils.RANDOM;
|
||||
import static org.apache.hudi.common.testutils.SchemaTestUtil.getSchemaFromResource;
|
||||
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;
|
||||
import static org.apache.hudi.io.storage.HoodieHFileReader.KEY_SCHEMA;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertNotNull;
|
||||
import static org.junit.jupiter.api.Assertions.assertNull;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
public class TestHoodieHFileReaderWriter {
|
||||
@TempDir File tempDir;
|
||||
private Path filePath;
|
||||
public class TestHoodieHFileReaderWriter extends TestHoodieReaderWriterBase {
|
||||
private static final String DUMMY_BASE_PATH = "dummy_base_path";
|
||||
// Number of records in HFile fixtures for compatibility tests
|
||||
private static final int NUM_RECORDS_FIXTURE = 50;
|
||||
private static final String SIMPLE_SCHEMA_HFILE_SUFFIX = "_simple.hfile";
|
||||
private static final String COMPLEX_SCHEMA_HFILE_SUFFIX = "_complex.hfile";
|
||||
private static final String BOOTSTRAP_INDEX_HFILE_SUFFIX = "_bootstrap_index_partitions.hfile";
|
||||
|
||||
@BeforeEach
|
||||
public void setup() throws IOException {
|
||||
filePath = new Path(tempDir.toString() + "tempFile.txt");
|
||||
@Override
|
||||
protected Path getFilePath() {
|
||||
return new Path(tempDir.toString() + "/f1_1-0-1_000.hfile");
|
||||
}
|
||||
|
||||
@AfterEach
|
||||
public void clearTempFile() {
|
||||
File file = new File(filePath.toString());
|
||||
if (file.exists()) {
|
||||
file.delete();
|
||||
}
|
||||
@Override
|
||||
protected HoodieFileWriter<GenericRecord> createWriter(
|
||||
Schema avroSchema, boolean populateMetaFields) throws Exception {
|
||||
String instantTime = "000";
|
||||
HoodieWriteConfig writeConfig = HoodieWriteConfig.newBuilder()
|
||||
.withPath(DUMMY_BASE_PATH)
|
||||
.withIndexConfig(HoodieIndexConfig.newBuilder()
|
||||
.bloomFilterNumEntries(1000).bloomFilterFPP(0.00001).build())
|
||||
.withPopulateMetaFields(populateMetaFields)
|
||||
.build();
|
||||
Configuration conf = new Configuration();
|
||||
TaskContextSupplier mockTaskContextSupplier = Mockito.mock(TaskContextSupplier.class);
|
||||
Supplier<Integer> partitionSupplier = Mockito.mock(Supplier.class);
|
||||
when(mockTaskContextSupplier.getPartitionIdSupplier()).thenReturn(partitionSupplier);
|
||||
when(partitionSupplier.get()).thenReturn(10);
|
||||
|
||||
return HoodieFileWriterFactory.newHFileFileWriter(
|
||||
instantTime, getFilePath(), writeConfig, avroSchema, conf, mockTaskContextSupplier);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected HoodieFileReader<GenericRecord> createReader(
|
||||
Configuration conf) throws Exception {
|
||||
CacheConfig cacheConfig = new CacheConfig(conf);
|
||||
return new HoodieHFileReader<>(conf, getFilePath(), cacheConfig, getFilePath().getFileSystem(conf));
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void verifyMetadata(Configuration conf) throws IOException {
|
||||
FileSystem fs = getFilePath().getFileSystem(conf);
|
||||
HFile.Reader hfileReader = HoodieHFileUtils.createHFileReader(fs, getFilePath(), new CacheConfig(conf), conf);
|
||||
assertEquals(HFILE_COMPARATOR.getClass(), hfileReader.getComparator().getClass());
|
||||
assertEquals(NUM_RECORDS, hfileReader.getEntries());
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void verifySchema(Configuration conf, String schemaPath) throws IOException {
|
||||
FileSystem fs = getFilePath().getFileSystem(conf);
|
||||
HFile.Reader hfileReader = HoodieHFileUtils.createHFileReader(fs, getFilePath(), new CacheConfig(conf), conf);
|
||||
assertEquals(getSchemaFromResource(TestHoodieHFileReaderWriter.class, schemaPath),
|
||||
new Schema.Parser().parse(new String(hfileReader.getHFileInfo().get(KEY_SCHEMA.getBytes()))));
|
||||
}
|
||||
|
||||
private static Stream<Arguments> populateMetaFieldsAndTestAvroWithMeta() {
|
||||
@@ -94,25 +136,11 @@ public class TestHoodieHFileReaderWriter {
|
||||
}).map(Arguments::of);
|
||||
}
|
||||
|
||||
private HoodieHFileWriter createHFileWriter(Schema avroSchema, boolean populateMetaFields) throws Exception {
|
||||
BloomFilter filter = BloomFilterFactory.createBloomFilter(1000, 0.00001, -1, BloomFilterTypeCode.SIMPLE.name());
|
||||
Configuration conf = new Configuration();
|
||||
TaskContextSupplier mockTaskContextSupplier = Mockito.mock(TaskContextSupplier.class);
|
||||
Supplier<Integer> partitionSupplier = Mockito.mock(Supplier.class);
|
||||
when(mockTaskContextSupplier.getPartitionIdSupplier()).thenReturn(partitionSupplier);
|
||||
when(partitionSupplier.get()).thenReturn(10);
|
||||
String instantTime = "000";
|
||||
|
||||
HoodieHFileConfig hoodieHFileConfig = new HoodieHFileConfig(conf, Compression.Algorithm.GZ, 1024 * 1024, 120 * 1024 * 1024,
|
||||
HoodieHFileReader.KEY_FIELD_NAME, PREFETCH_ON_OPEN, CACHE_DATA_IN_L1, DROP_BEHIND_CACHE_COMPACTION, filter, HFILE_COMPARATOR);
|
||||
return new HoodieHFileWriter(instantTime, filePath, hoodieHFileConfig, avroSchema, mockTaskContextSupplier, populateMetaFields);
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@MethodSource("populateMetaFieldsAndTestAvroWithMeta")
|
||||
public void testWriteReadHFile(boolean populateMetaFields, boolean testAvroWithMeta) throws Exception {
|
||||
public void testWriteReadHFileWithMetaFields(boolean populateMetaFields, boolean testAvroWithMeta) throws Exception {
|
||||
Schema avroSchema = getSchemaFromResource(TestHoodieOrcReaderWriter.class, "/exampleSchemaWithMetaFields.avsc");
|
||||
HoodieHFileWriter writer = createHFileWriter(avroSchema, populateMetaFields);
|
||||
HoodieFileWriter<GenericRecord> writer = createWriter(avroSchema, populateMetaFields);
|
||||
List<String> keys = new ArrayList<>();
|
||||
Map<String, GenericRecord> recordMap = new HashMap<>();
|
||||
for (int i = 0; i < 100; i++) {
|
||||
@@ -134,8 +162,7 @@ public class TestHoodieHFileReaderWriter {
|
||||
writer.close();
|
||||
|
||||
Configuration conf = new Configuration();
|
||||
CacheConfig cacheConfig = new CacheConfig(conf);
|
||||
HoodieHFileReader hoodieHFileReader = new HoodieHFileReader(conf, filePath, cacheConfig, filePath.getFileSystem(conf));
|
||||
HoodieHFileReader hoodieHFileReader = (HoodieHFileReader) createReader(conf);
|
||||
List<Pair<String, IndexedRecord>> records = hoodieHFileReader.readAllRecords();
|
||||
records.forEach(entry -> assertEquals(entry.getSecond(), recordMap.get(entry.getFirst())));
|
||||
hoodieHFileReader.close();
|
||||
@@ -145,7 +172,7 @@ public class TestHoodieHFileReaderWriter {
|
||||
Set<String> rowsToFetch = getRandomKeys(randomRowstoFetch, keys);
|
||||
List<String> rowsList = new ArrayList<>(rowsToFetch);
|
||||
Collections.sort(rowsList);
|
||||
hoodieHFileReader = new HoodieHFileReader(conf, filePath, cacheConfig, filePath.getFileSystem(conf));
|
||||
hoodieHFileReader = (HoodieHFileReader) createReader(conf);
|
||||
List<Pair<String, GenericRecord>> result = hoodieHFileReader.readRecords(rowsList);
|
||||
assertEquals(result.size(), randomRowstoFetch);
|
||||
result.forEach(entry -> {
|
||||
@@ -160,6 +187,90 @@ public class TestHoodieHFileReaderWriter {
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
@Test
|
||||
public void testWriteReadWithEvolvedSchema() throws Exception {
|
||||
// Disable the test with evolved schema for HFile since it's not supported
|
||||
// TODO(HUDI-3683): fix the schema evolution for HFile
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testReadHFileFormatRecords() throws Exception {
|
||||
writeFileWithSimpleSchema();
|
||||
FileSystem fs = FSUtils.getFs(getFilePath().toString(), new Configuration());
|
||||
byte[] content = FileIOUtils.readAsByteArray(
|
||||
fs.open(getFilePath()), (int) fs.getFileStatus(getFilePath()).getLen());
|
||||
// Reading byte array in HFile format, without actual file path
|
||||
HoodieHFileReader<GenericRecord> hfileReader =
|
||||
new HoodieHFileReader<>(fs, new Path(DUMMY_BASE_PATH), content);
|
||||
Schema avroSchema = getSchemaFromResource(TestHoodieReaderWriterBase.class, "/exampleSchema.avsc");
|
||||
assertEquals(NUM_RECORDS, hfileReader.getTotalRecords());
|
||||
verifySimpleRecords(hfileReader.getRecordIterator(avroSchema));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testReaderGetRecordIterator() throws Exception {
|
||||
writeFileWithSimpleSchema();
|
||||
HoodieHFileReader<GenericRecord> hfileReader =
|
||||
(HoodieHFileReader<GenericRecord>) createReader(new Configuration());
|
||||
List<String> keys =
|
||||
IntStream.concat(IntStream.range(40, NUM_RECORDS * 2), IntStream.range(10, 20))
|
||||
.mapToObj(i -> "key" + String.format("%02d", i)).collect(Collectors.toList());
|
||||
Schema avroSchema = getSchemaFromResource(TestHoodieReaderWriterBase.class, "/exampleSchema.avsc");
|
||||
Iterator<GenericRecord> iterator = hfileReader.getRecordIterator(keys, avroSchema);
|
||||
|
||||
List<Integer> expectedIds =
|
||||
IntStream.concat(IntStream.range(40, NUM_RECORDS), IntStream.range(10, 20))
|
||||
.boxed().collect(Collectors.toList());
|
||||
int index = 0;
|
||||
while (iterator.hasNext()) {
|
||||
GenericRecord record = iterator.next();
|
||||
String key = "key" + String.format("%02d", expectedIds.get(index));
|
||||
assertEquals(key, record.get("_row_key").toString());
|
||||
assertEquals(Integer.toString(expectedIds.get(index)), record.get("time").toString());
|
||||
assertEquals(expectedIds.get(index), record.get("number"));
|
||||
index++;
|
||||
}
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@ValueSource(strings = {
|
||||
"/hudi_0_9_hbase_1_2_3", "/hudi_0_10_hbase_1_2_3", "/hudi_0_11_hbase_2_4_9"})
|
||||
public void testHoodieHFileCompatibility(String hfilePrefix) throws IOException {
|
||||
// This fixture is generated from TestHoodieReaderWriterBase#testWriteReadPrimitiveRecord()
|
||||
// using different Hudi releases
|
||||
String simpleHFile = hfilePrefix + SIMPLE_SCHEMA_HFILE_SUFFIX;
|
||||
// This fixture is generated from TestHoodieReaderWriterBase#testWriteReadComplexRecord()
|
||||
// using different Hudi releases
|
||||
String complexHFile = hfilePrefix + COMPLEX_SCHEMA_HFILE_SUFFIX;
|
||||
// This fixture is generated from TestBootstrapIndex#testBootstrapIndex()
|
||||
// using different Hudi releases. The file is copied from .hoodie/.aux/.bootstrap/.partitions/
|
||||
String bootstrapIndexFile = hfilePrefix + BOOTSTRAP_INDEX_HFILE_SUFFIX;
|
||||
|
||||
FileSystem fs = FSUtils.getFs(getFilePath().toString(), new Configuration());
|
||||
byte[] content = readHFileFromResources(simpleHFile);
|
||||
verifyHFileReader(
|
||||
HoodieHFileUtils.createHFileReader(fs, new Path(DUMMY_BASE_PATH), content),
|
||||
hfilePrefix, true, HFILE_COMPARATOR.getClass(), NUM_RECORDS_FIXTURE);
|
||||
HoodieHFileReader<GenericRecord> hfileReader =
|
||||
new HoodieHFileReader<>(fs, new Path(DUMMY_BASE_PATH), content);
|
||||
Schema avroSchema = getSchemaFromResource(TestHoodieReaderWriterBase.class, "/exampleSchema.avsc");
|
||||
assertEquals(NUM_RECORDS_FIXTURE, hfileReader.getTotalRecords());
|
||||
verifySimpleRecords(hfileReader.getRecordIterator(avroSchema));
|
||||
|
||||
content = readHFileFromResources(complexHFile);
|
||||
verifyHFileReader(HoodieHFileUtils.createHFileReader(fs, new Path(DUMMY_BASE_PATH), content),
|
||||
hfilePrefix, true, HFILE_COMPARATOR.getClass(), NUM_RECORDS_FIXTURE);
|
||||
hfileReader = new HoodieHFileReader<>(fs, new Path(DUMMY_BASE_PATH), content);
|
||||
avroSchema = getSchemaFromResource(TestHoodieReaderWriterBase.class, "/exampleSchemaWithUDT.avsc");
|
||||
assertEquals(NUM_RECORDS_FIXTURE, hfileReader.getTotalRecords());
|
||||
verifySimpleRecords(hfileReader.getRecordIterator(avroSchema));
|
||||
|
||||
content = readHFileFromResources(bootstrapIndexFile);
|
||||
verifyHFileReader(HoodieHFileUtils.createHFileReader(fs, new Path(DUMMY_BASE_PATH), content),
|
||||
hfilePrefix, false, HFileBootstrapIndex.HoodieKVComparator.class, 4);
|
||||
}
|
||||
|
||||
private Set<String> getRandomKeys(int count, List<String> keys) {
|
||||
Set<String> rowKeys = new HashSet<>();
|
||||
int totalKeys = keys.size();
|
||||
@@ -171,4 +282,26 @@ public class TestHoodieHFileReaderWriter {
|
||||
}
|
||||
return rowKeys;
|
||||
}
|
||||
|
||||
private byte[] readHFileFromResources(String filename) throws IOException {
|
||||
long size = TestHoodieHFileReaderWriter.class
|
||||
.getResource(filename).openConnection().getContentLength();
|
||||
return FileIOUtils.readAsByteArray(
|
||||
TestHoodieHFileReaderWriter.class.getResourceAsStream(filename), (int) size);
|
||||
}
|
||||
|
||||
private void verifyHFileReader(
|
||||
HFile.Reader reader, String hfileName, boolean mayUseDefaultComparator,
|
||||
Class<?> clazz, int count) {
|
||||
// HFile version is 3
|
||||
assertEquals(3, reader.getTrailer().getMajorVersion());
|
||||
if (mayUseDefaultComparator && hfileName.contains("hudi_0_9")) {
|
||||
// Pre Hudi 0.10, the default comparator is used for metadata table HFiles
|
||||
// For bootstrap index HFiles, the custom comparator is always used
|
||||
assertEquals(CellComparatorImpl.class, reader.getComparator().getClass());
|
||||
} else {
|
||||
assertEquals(clazz, reader.getComparator().getClass());
|
||||
}
|
||||
assertEquals(count, reader.getEntries());
|
||||
}
|
||||
}
|
||||
|
||||
@@ -18,53 +18,40 @@
|
||||
|
||||
package org.apache.hudi.io.storage;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericData;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.common.bloom.BloomFilter;
|
||||
import org.apache.hudi.common.bloom.BloomFilterFactory;
|
||||
import org.apache.hudi.common.bloom.BloomFilterTypeCode;
|
||||
import org.apache.hudi.common.engine.TaskContextSupplier;
|
||||
import org.apache.hudi.config.HoodieStorageConfig;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.orc.CompressionKind;
|
||||
import org.apache.orc.OrcFile;
|
||||
import org.apache.orc.Reader;
|
||||
import org.junit.jupiter.api.AfterEach;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.mockito.Mockito;
|
||||
|
||||
import java.io.File;
|
||||
import java.util.Collections;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.io.IOException;
|
||||
|
||||
import static org.apache.hudi.avro.HoodieAvroWriteSupport.HOODIE_AVRO_BLOOM_FILTER_METADATA_KEY;
|
||||
import static org.apache.hudi.avro.HoodieAvroWriteSupport.HOODIE_MAX_RECORD_KEY_FOOTER;
|
||||
import static org.apache.hudi.avro.HoodieAvroWriteSupport.HOODIE_MIN_RECORD_KEY_FOOTER;
|
||||
import static org.apache.hudi.common.testutils.SchemaTestUtil.getSchemaFromResource;
|
||||
import static org.apache.hudi.io.storage.HoodieOrcConfig.AVRO_SCHEMA_METADATA_KEY;
|
||||
import static org.junit.jupiter.api.Assertions.assertFalse;
|
||||
import static org.junit.jupiter.api.Assertions.assertNull;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
|
||||
public class TestHoodieOrcReaderWriter {
|
||||
private final Path filePath = new Path(System.getProperty("java.io.tmpdir") + "/f1_1-0-1_000.orc");
|
||||
public class TestHoodieOrcReaderWriter extends TestHoodieReaderWriterBase {
|
||||
|
||||
@BeforeEach
|
||||
@AfterEach
|
||||
public void clearTempFile() {
|
||||
File file = new File(filePath.toString());
|
||||
if (file.exists()) {
|
||||
file.delete();
|
||||
}
|
||||
@Override
|
||||
protected Path getFilePath() {
|
||||
return new Path(tempDir.toString() + "/f1_1-0-1_000.orc");
|
||||
}
|
||||
|
||||
private HoodieOrcWriter createOrcWriter(Schema avroSchema) throws Exception {
|
||||
@Override
|
||||
protected HoodieFileWriter<GenericRecord> createWriter(
|
||||
Schema avroSchema, boolean populateMetaFields) throws Exception {
|
||||
BloomFilter filter = BloomFilterFactory.createBloomFilter(1000, 0.00001, -1, BloomFilterTypeCode.SIMPLE.name());
|
||||
Configuration conf = new Configuration();
|
||||
int orcStripSize = Integer.parseInt(HoodieStorageConfig.ORC_STRIPE_SIZE.defaultValue());
|
||||
@@ -73,189 +60,41 @@ public class TestHoodieOrcReaderWriter {
|
||||
HoodieOrcConfig config = new HoodieOrcConfig(conf, CompressionKind.ZLIB, orcStripSize, orcBlockSize, maxFileSize, filter);
|
||||
TaskContextSupplier mockTaskContextSupplier = Mockito.mock(TaskContextSupplier.class);
|
||||
String instantTime = "000";
|
||||
return new HoodieOrcWriter(instantTime, filePath, config, avroSchema, mockTaskContextSupplier);
|
||||
return new HoodieOrcWriter<>(instantTime, getFilePath(), config, avroSchema, mockTaskContextSupplier);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testWriteReadMetadata() throws Exception {
|
||||
Schema avroSchema = getSchemaFromResource(TestHoodieOrcReaderWriter.class, "/exampleSchema.avsc");
|
||||
HoodieOrcWriter writer = createOrcWriter(avroSchema);
|
||||
for (int i = 0; i < 3; i++) {
|
||||
GenericRecord record = new GenericData.Record(avroSchema);
|
||||
record.put("_row_key", "key" + i);
|
||||
record.put("time", Integer.toString(i));
|
||||
record.put("number", i);
|
||||
writer.writeAvro("key" + i, record);
|
||||
}
|
||||
writer.close();
|
||||
@Override
|
||||
protected HoodieFileReader<GenericRecord> createReader(
|
||||
Configuration conf) throws Exception {
|
||||
return HoodieFileReaderFactory.getFileReader(conf, getFilePath());
|
||||
}
|
||||
|
||||
Configuration conf = new Configuration();
|
||||
Reader orcReader = OrcFile.createReader(filePath, OrcFile.readerOptions(conf));
|
||||
@Override
|
||||
protected void verifyMetadata(Configuration conf) throws IOException {
|
||||
Reader orcReader = OrcFile.createReader(getFilePath(), OrcFile.readerOptions(conf));
|
||||
assertEquals(4, orcReader.getMetadataKeys().size());
|
||||
assertTrue(orcReader.getMetadataKeys().contains(HOODIE_MIN_RECORD_KEY_FOOTER));
|
||||
assertTrue(orcReader.getMetadataKeys().contains(HOODIE_MAX_RECORD_KEY_FOOTER));
|
||||
assertTrue(orcReader.getMetadataKeys().contains(HOODIE_AVRO_BLOOM_FILTER_METADATA_KEY));
|
||||
assertTrue(orcReader.getMetadataKeys().contains(AVRO_SCHEMA_METADATA_KEY));
|
||||
assertEquals(CompressionKind.ZLIB.name(), orcReader.getCompressionKind().toString());
|
||||
|
||||
HoodieFileReader<GenericRecord> hoodieReader = HoodieFileReaderFactory.getFileReader(conf, filePath);
|
||||
BloomFilter filter = hoodieReader.readBloomFilter();
|
||||
for (int i = 0; i < 3; i++) {
|
||||
assertTrue(filter.mightContain("key" + i));
|
||||
}
|
||||
assertFalse(filter.mightContain("non-existent-key"));
|
||||
assertEquals(3, hoodieReader.getTotalRecords());
|
||||
String[] minMaxRecordKeys = hoodieReader.readMinMaxRecordKeys();
|
||||
assertEquals(2, minMaxRecordKeys.length);
|
||||
assertEquals("key0", minMaxRecordKeys[0]);
|
||||
assertEquals("key2", minMaxRecordKeys[1]);
|
||||
assertEquals(NUM_RECORDS, orcReader.getNumberOfRows());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testWriteReadPrimitiveRecord() throws Exception {
|
||||
Schema avroSchema = getSchemaFromResource(TestHoodieOrcReaderWriter.class, "/exampleSchema.avsc");
|
||||
HoodieOrcWriter writer = createOrcWriter(avroSchema);
|
||||
for (int i = 0; i < 3; i++) {
|
||||
GenericRecord record = new GenericData.Record(avroSchema);
|
||||
record.put("_row_key", "key" + i);
|
||||
record.put("time", Integer.toString(i));
|
||||
record.put("number", i);
|
||||
writer.writeAvro("key" + i, record);
|
||||
}
|
||||
writer.close();
|
||||
|
||||
Configuration conf = new Configuration();
|
||||
Reader orcReader = OrcFile.createReader(filePath, OrcFile.readerOptions(conf));
|
||||
assertEquals("struct<_row_key:string,time:string,number:int>", orcReader.getSchema().toString());
|
||||
assertEquals(3, orcReader.getNumberOfRows());
|
||||
|
||||
HoodieFileReader<GenericRecord> hoodieReader = HoodieFileReaderFactory.getFileReader(conf, filePath);
|
||||
Iterator<GenericRecord> iter = hoodieReader.getRecordIterator();
|
||||
int index = 0;
|
||||
while (iter.hasNext()) {
|
||||
GenericRecord record = iter.next();
|
||||
assertEquals("key" + index, record.get("_row_key").toString());
|
||||
assertEquals(Integer.toString(index), record.get("time").toString());
|
||||
assertEquals(index, record.get("number"));
|
||||
index++;
|
||||
@Override
|
||||
protected void verifySchema(Configuration conf, String schemaPath) throws IOException {
|
||||
Reader orcReader = OrcFile.createReader(getFilePath(), OrcFile.readerOptions(conf));
|
||||
if ("/exampleSchema.avsc".equals(schemaPath)) {
|
||||
assertEquals("struct<_row_key:string,time:string,number:int>",
|
||||
orcReader.getSchema().toString());
|
||||
} else if ("/exampleSchemaWithUDT.avsc".equals(schemaPath)) {
|
||||
assertEquals("struct<_row_key:string,time:string,number:int,driver:struct<driver_name:string,list:array<int>,map:map<string,string>>>",
|
||||
orcReader.getSchema().toString());
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testWriteReadComplexRecord() throws Exception {
|
||||
Schema avroSchema = getSchemaFromResource(TestHoodieOrcReaderWriter.class, "/exampleSchemaWithUDT.avsc");
|
||||
Schema udtSchema = avroSchema.getField("driver").schema().getTypes().get(1);
|
||||
HoodieOrcWriter writer = createOrcWriter(avroSchema);
|
||||
for (int i = 0; i < 3; i++) {
|
||||
GenericRecord record = new GenericData.Record(avroSchema);
|
||||
record.put("_row_key", "key" + i);
|
||||
record.put("time", Integer.toString(i));
|
||||
record.put("number", i);
|
||||
GenericRecord innerRecord = new GenericData.Record(udtSchema);
|
||||
innerRecord.put("driver_name", "driver" + i);
|
||||
innerRecord.put("list", Collections.singletonList(i));
|
||||
innerRecord.put("map", Collections.singletonMap("key" + i, "value" + i));
|
||||
record.put("driver", innerRecord);
|
||||
writer.writeAvro("key" + i, record);
|
||||
}
|
||||
writer.close();
|
||||
|
||||
Configuration conf = new Configuration();
|
||||
Reader reader = OrcFile.createReader(filePath, OrcFile.readerOptions(conf));
|
||||
assertEquals("struct<_row_key:string,time:string,number:int,driver:struct<driver_name:string,list:array<int>,map:map<string,string>>>",
|
||||
reader.getSchema().toString());
|
||||
assertEquals(3, reader.getNumberOfRows());
|
||||
|
||||
HoodieFileReader<GenericRecord> hoodieReader = HoodieFileReaderFactory.getFileReader(conf, filePath);
|
||||
Iterator<GenericRecord> iter = hoodieReader.getRecordIterator();
|
||||
int index = 0;
|
||||
while (iter.hasNext()) {
|
||||
GenericRecord record = iter.next();
|
||||
assertEquals("key" + index, record.get("_row_key").toString());
|
||||
assertEquals(Integer.toString(index), record.get("time").toString());
|
||||
assertEquals(index, record.get("number"));
|
||||
GenericRecord innerRecord = (GenericRecord) record.get("driver");
|
||||
assertEquals("driver" + index, innerRecord.get("driver_name").toString());
|
||||
assertEquals(1, ((List<?>)innerRecord.get("list")).size());
|
||||
assertEquals(index, ((List<?>)innerRecord.get("list")).get(0));
|
||||
assertEquals("value" + index, ((Map<?,?>)innerRecord.get("map")).get("key" + index).toString());
|
||||
index++;
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testWriteReadWithEvolvedSchema() throws Exception {
|
||||
Schema avroSchema = getSchemaFromResource(TestHoodieOrcReaderWriter.class, "/exampleSchema.avsc");
|
||||
HoodieOrcWriter writer = createOrcWriter(avroSchema);
|
||||
for (int i = 0; i < 3; i++) {
|
||||
GenericRecord record = new GenericData.Record(avroSchema);
|
||||
record.put("_row_key", "key" + i);
|
||||
record.put("time", Integer.toString(i));
|
||||
record.put("number", i);
|
||||
writer.writeAvro("key" + i, record);
|
||||
}
|
||||
writer.close();
|
||||
|
||||
Configuration conf = new Configuration();
|
||||
HoodieFileReader<GenericRecord> hoodieReader = HoodieFileReaderFactory.getFileReader(conf, filePath);
|
||||
Schema evolvedSchema = getSchemaFromResource(TestHoodieOrcReaderWriter.class, "/exampleEvolvedSchema.avsc");
|
||||
Iterator<GenericRecord> iter = hoodieReader.getRecordIterator(evolvedSchema);
|
||||
int index = 0;
|
||||
while (iter.hasNext()) {
|
||||
GenericRecord record = iter.next();
|
||||
assertEquals("key" + index, record.get("_row_key").toString());
|
||||
assertEquals(Integer.toString(index), record.get("time").toString());
|
||||
assertEquals(index, record.get("number"));
|
||||
assertNull(record.get("added_field"));
|
||||
index++;
|
||||
}
|
||||
|
||||
evolvedSchema = getSchemaFromResource(TestHoodieOrcReaderWriter.class, "/exampleEvolvedSchemaChangeOrder.avsc");
|
||||
iter = hoodieReader.getRecordIterator(evolvedSchema);
|
||||
index = 0;
|
||||
while (iter.hasNext()) {
|
||||
GenericRecord record = iter.next();
|
||||
assertEquals("key" + index, record.get("_row_key").toString());
|
||||
assertEquals(Integer.toString(index), record.get("time").toString());
|
||||
assertEquals(index, record.get("number"));
|
||||
assertNull(record.get("added_field"));
|
||||
index++;
|
||||
}
|
||||
|
||||
evolvedSchema = getSchemaFromResource(TestHoodieOrcReaderWriter.class, "/exampleEvolvedSchemaColumnRequire.avsc");
|
||||
iter = hoodieReader.getRecordIterator(evolvedSchema);
|
||||
index = 0;
|
||||
while (iter.hasNext()) {
|
||||
GenericRecord record = iter.next();
|
||||
assertEquals("key" + index, record.get("_row_key").toString());
|
||||
assertEquals(Integer.toString(index), record.get("time").toString());
|
||||
assertEquals(index, record.get("number"));
|
||||
assertNull(record.get("added_field"));
|
||||
index++;
|
||||
}
|
||||
|
||||
evolvedSchema = getSchemaFromResource(TestHoodieOrcReaderWriter.class, "/exampleEvolvedSchemaColumnType.avsc");
|
||||
iter = hoodieReader.getRecordIterator(evolvedSchema);
|
||||
index = 0;
|
||||
while (iter.hasNext()) {
|
||||
GenericRecord record = iter.next();
|
||||
assertEquals("key" + index, record.get("_row_key").toString());
|
||||
assertEquals(Integer.toString(index), record.get("time").toString());
|
||||
assertEquals(Integer.toString(index), record.get("number").toString());
|
||||
assertNull(record.get("added_field"));
|
||||
index++;
|
||||
}
|
||||
|
||||
evolvedSchema = getSchemaFromResource(TestHoodieOrcReaderWriter.class, "/exampleEvolvedSchemaDeleteColumn.avsc");
|
||||
iter = hoodieReader.getRecordIterator(evolvedSchema);
|
||||
index = 0;
|
||||
while (iter.hasNext()) {
|
||||
GenericRecord record = iter.next();
|
||||
assertEquals("key" + index, record.get("_row_key").toString());
|
||||
assertEquals(Integer.toString(index), record.get("time").toString());
|
||||
assertNull(record.get("number"));
|
||||
assertNull(record.get("added_field"));
|
||||
index++;
|
||||
}
|
||||
@Override
|
||||
public void testReaderFilterRowKeys() {
|
||||
// TODO(HUDI-3682): fix filterRowKeys test for ORC due to a bug in ORC logic
|
||||
}
|
||||
}
|
||||
|
||||
@@ -0,0 +1,250 @@
|
||||
/*
|
||||
* 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.hudi.common.bloom.BloomFilter;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericData;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.junit.jupiter.api.AfterEach;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.api.io.TempDir;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.util.Collections;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.IntStream;
|
||||
|
||||
import static org.apache.hudi.common.testutils.SchemaTestUtil.getSchemaFromResource;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertFalse;
|
||||
import static org.junit.jupiter.api.Assertions.assertNull;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
|
||||
/**
|
||||
* Abstract class for unit tests of {@link HoodieFileReader} and {@link HoodieFileWriter}
|
||||
* for different file format
|
||||
*/
|
||||
public abstract class TestHoodieReaderWriterBase {
|
||||
protected static final int NUM_RECORDS = 50;
|
||||
@TempDir
|
||||
protected File tempDir;
|
||||
|
||||
protected abstract Path getFilePath();
|
||||
|
||||
protected abstract HoodieFileWriter<GenericRecord> createWriter(
|
||||
Schema avroSchema, boolean populateMetaFields) throws Exception;
|
||||
|
||||
protected abstract HoodieFileReader<GenericRecord> createReader(
|
||||
Configuration conf) throws Exception;
|
||||
|
||||
protected abstract void verifyMetadata(Configuration conf) throws IOException;
|
||||
|
||||
protected abstract void verifySchema(Configuration conf, String schemaPath) throws IOException;
|
||||
|
||||
@BeforeEach
|
||||
@AfterEach
|
||||
public void clearTempFile() {
|
||||
File file = new File(getFilePath().toString());
|
||||
if (file.exists()) {
|
||||
file.delete();
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testWriteReadMetadata() throws Exception {
|
||||
Schema avroSchema = getSchemaFromResource(TestHoodieReaderWriterBase.class, "/exampleSchema.avsc");
|
||||
writeFileWithSimpleSchema();
|
||||
|
||||
Configuration conf = new Configuration();
|
||||
verifyMetadata(conf);
|
||||
|
||||
HoodieFileReader<GenericRecord> hoodieReader = createReader(conf);
|
||||
BloomFilter filter = hoodieReader.readBloomFilter();
|
||||
for (int i = 0; i < NUM_RECORDS; i++) {
|
||||
String key = "key" + String.format("%02d", i);
|
||||
assertTrue(filter.mightContain(key));
|
||||
}
|
||||
assertFalse(filter.mightContain("non-existent-key"));
|
||||
assertEquals(avroSchema, hoodieReader.getSchema());
|
||||
assertEquals(NUM_RECORDS, hoodieReader.getTotalRecords());
|
||||
String[] minMaxRecordKeys = hoodieReader.readMinMaxRecordKeys();
|
||||
assertEquals(2, minMaxRecordKeys.length);
|
||||
assertEquals("key00", minMaxRecordKeys[0]);
|
||||
assertEquals("key" + (NUM_RECORDS - 1), minMaxRecordKeys[1]);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testWriteReadPrimitiveRecord() throws Exception {
|
||||
String schemaPath = "/exampleSchema.avsc";
|
||||
writeFileWithSimpleSchema();
|
||||
|
||||
Configuration conf = new Configuration();
|
||||
verifyMetadata(conf);
|
||||
verifySchema(conf, schemaPath);
|
||||
verifySimpleRecords(createReader(conf).getRecordIterator());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testWriteReadComplexRecord() throws Exception {
|
||||
String schemaPath = "/exampleSchemaWithUDT.avsc";
|
||||
Schema avroSchema = getSchemaFromResource(TestHoodieReaderWriterBase.class, schemaPath);
|
||||
Schema udtSchema = avroSchema.getField("driver").schema().getTypes().get(1);
|
||||
HoodieFileWriter<GenericRecord> writer = createWriter(avroSchema, true);
|
||||
for (int i = 0; i < NUM_RECORDS; i++) {
|
||||
GenericRecord record = new GenericData.Record(avroSchema);
|
||||
String key = "key" + String.format("%02d", i);
|
||||
record.put("_row_key", key);
|
||||
record.put("time", Integer.toString(i));
|
||||
record.put("number", i);
|
||||
GenericRecord innerRecord = new GenericData.Record(udtSchema);
|
||||
innerRecord.put("driver_name", "driver" + i);
|
||||
innerRecord.put("list", Collections.singletonList(i));
|
||||
innerRecord.put("map", Collections.singletonMap(key, "value" + i));
|
||||
record.put("driver", innerRecord);
|
||||
writer.writeAvro(key, record);
|
||||
}
|
||||
writer.close();
|
||||
|
||||
Configuration conf = new Configuration();
|
||||
verifyMetadata(conf);
|
||||
verifySchema(conf, schemaPath);
|
||||
verifyComplexRecords(createReader(conf).getRecordIterator());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testWriteReadWithEvolvedSchema() throws Exception {
|
||||
writeFileWithSimpleSchema();
|
||||
|
||||
Configuration conf = new Configuration();
|
||||
HoodieFileReader<GenericRecord> hoodieReader = createReader(conf);
|
||||
String[] schemaList = new String[] {
|
||||
"/exampleEvolvedSchema.avsc", "/exampleEvolvedSchemaChangeOrder.avsc",
|
||||
"/exampleEvolvedSchemaColumnRequire.avsc", "/exampleEvolvedSchemaColumnType.avsc",
|
||||
"/exampleEvolvedSchemaDeleteColumn.avsc"};
|
||||
|
||||
for (String evolvedSchemaPath : schemaList) {
|
||||
verifyReaderWithSchema(evolvedSchemaPath, hoodieReader);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testReaderFilterRowKeys() throws Exception {
|
||||
writeFileWithSimpleSchema();
|
||||
Configuration conf = new Configuration();
|
||||
verifyMetadata(conf);
|
||||
verifyFilterRowKeys(createReader(conf));
|
||||
}
|
||||
|
||||
protected void writeFileWithSimpleSchema() throws Exception {
|
||||
Schema avroSchema = getSchemaFromResource(TestHoodieReaderWriterBase.class, "/exampleSchema.avsc");
|
||||
HoodieFileWriter<GenericRecord> writer = createWriter(avroSchema, true);
|
||||
for (int i = 0; i < NUM_RECORDS; i++) {
|
||||
GenericRecord record = new GenericData.Record(avroSchema);
|
||||
String key = "key" + String.format("%02d", i);
|
||||
record.put("_row_key", key);
|
||||
record.put("time", Integer.toString(i));
|
||||
record.put("number", i);
|
||||
writer.writeAvro(key, record);
|
||||
}
|
||||
writer.close();
|
||||
}
|
||||
|
||||
protected void verifySimpleRecords(Iterator<GenericRecord> iterator) {
|
||||
int index = 0;
|
||||
while (iterator.hasNext()) {
|
||||
GenericRecord record = iterator.next();
|
||||
String key = "key" + String.format("%02d", index);
|
||||
assertEquals(key, record.get("_row_key").toString());
|
||||
assertEquals(Integer.toString(index), record.get("time").toString());
|
||||
assertEquals(index, record.get("number"));
|
||||
index++;
|
||||
}
|
||||
}
|
||||
|
||||
protected void verifyComplexRecords(Iterator<GenericRecord> iterator) {
|
||||
int index = 0;
|
||||
while (iterator.hasNext()) {
|
||||
GenericRecord record = iterator.next();
|
||||
String key = "key" + String.format("%02d", index);
|
||||
assertEquals(key, record.get("_row_key").toString());
|
||||
assertEquals(Integer.toString(index), record.get("time").toString());
|
||||
assertEquals(index, record.get("number"));
|
||||
GenericRecord innerRecord = (GenericRecord) record.get("driver");
|
||||
assertEquals("driver" + index, innerRecord.get("driver_name").toString());
|
||||
assertEquals(1, ((List<?>) innerRecord.get("list")).size());
|
||||
assertEquals(index, ((List<?>) innerRecord.get("list")).get(0));
|
||||
Map<?, ?> mapping = (Map<?, ?>) innerRecord.get("map");
|
||||
boolean match = false;
|
||||
for (Object innerKey : mapping.keySet()) {
|
||||
// The innerKey may not be in the type of String, so we have to
|
||||
// use the following logic for validation
|
||||
if (innerKey.toString().equals(key)) {
|
||||
assertEquals("value" + index, mapping.get(innerKey).toString());
|
||||
match = true;
|
||||
}
|
||||
}
|
||||
assertTrue(match);
|
||||
index++;
|
||||
}
|
||||
}
|
||||
|
||||
private void verifyFilterRowKeys(HoodieFileReader<GenericRecord> hoodieReader) {
|
||||
Set<String> candidateRowKeys = IntStream.range(40, NUM_RECORDS * 2)
|
||||
.mapToObj(i -> "key" + String.format("%02d", i)).collect(Collectors.toSet());
|
||||
List<String> expectedKeys = IntStream.range(40, NUM_RECORDS)
|
||||
.mapToObj(i -> "key" + String.format("%02d", i)).sorted().collect(Collectors.toList());
|
||||
assertEquals(expectedKeys, hoodieReader.filterRowKeys(candidateRowKeys)
|
||||
.stream().sorted().collect(Collectors.toList()));
|
||||
}
|
||||
|
||||
private void verifyReaderWithSchema(String schemaPath, HoodieFileReader<GenericRecord> hoodieReader) throws IOException {
|
||||
Schema evolvedSchema = getSchemaFromResource(TestHoodieReaderWriterBase.class, schemaPath);
|
||||
Iterator<GenericRecord> iter = hoodieReader.getRecordIterator(evolvedSchema);
|
||||
int index = 0;
|
||||
while (iter.hasNext()) {
|
||||
verifyRecord(schemaPath, iter.next(), index);
|
||||
index++;
|
||||
}
|
||||
}
|
||||
|
||||
private void verifyRecord(String schemaPath, GenericRecord record, int index) {
|
||||
String numStr = String.format("%02d", index);
|
||||
assertEquals("key" + numStr, record.get("_row_key").toString());
|
||||
assertEquals(Integer.toString(index), record.get("time").toString());
|
||||
if ("/exampleEvolvedSchemaColumnType.avsc".equals(schemaPath)) {
|
||||
assertEquals(Integer.toString(index), record.get("number").toString());
|
||||
} else if ("/exampleEvolvedSchemaDeleteColumn.avsc".equals(schemaPath)) {
|
||||
assertNull(record.get("number"));
|
||||
} else {
|
||||
assertEquals(index, record.get("number"));
|
||||
}
|
||||
assertNull(record.get("added_field"));
|
||||
}
|
||||
}
|
||||
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
@@ -110,6 +110,12 @@
|
||||
</exclusion>
|
||||
</exclusions>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.zookeeper</groupId>
|
||||
<artifactId>zookeeper</artifactId>
|
||||
<version>${zookeeper.version}</version>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
|
||||
<!-- Hive - Tests -->
|
||||
<dependency>
|
||||
|
||||
@@ -38,6 +38,7 @@ import org.apache.hudi.common.table.view.TableFileSystemView.BaseFileOnlyView;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.io.storage.HoodieHFileUtils;
|
||||
import org.apache.hudi.timeline.service.TimelineService;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
@@ -66,6 +67,8 @@ import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
import static org.apache.hudi.io.storage.HoodieHFileReader.KEY_SCHEMA;
|
||||
|
||||
/**
|
||||
* Utility methods to aid testing inside the HoodieClient module.
|
||||
*/
|
||||
@@ -241,9 +244,10 @@ public class HoodieClientTestUtils {
|
||||
Schema schema = null;
|
||||
for (String path : paths) {
|
||||
try {
|
||||
HFile.Reader reader = HFile.createReader(fs, new Path(path), cacheConfig, fs.getConf());
|
||||
HFile.Reader reader =
|
||||
HoodieHFileUtils.createHFileReader(fs, new Path(path), cacheConfig, fs.getConf());
|
||||
if (schema == null) {
|
||||
schema = new Schema.Parser().parse(new String(reader.loadFileInfo().get("schema".getBytes())));
|
||||
schema = new Schema.Parser().parse(new String(reader.getHFileInfo().get(KEY_SCHEMA.getBytes())));
|
||||
}
|
||||
HFileScanner scanner = reader.getScanner(false, false);
|
||||
if (!scanner.seekTo()) {
|
||||
@@ -252,7 +256,7 @@ public class HoodieClientTestUtils {
|
||||
}
|
||||
|
||||
do {
|
||||
Cell c = scanner.getKeyValue();
|
||||
Cell c = scanner.getCell();
|
||||
byte[] value = Arrays.copyOfRange(c.getValueArray(), c.getValueOffset(), c.getValueOffset() + c.getValueLength());
|
||||
valuesAsList.add(HoodieAvroUtils.bytesToAvro(value, schema));
|
||||
} while (scanner.next());
|
||||
|
||||
@@ -221,14 +221,13 @@
|
||||
<groupId>org.apache.hbase</groupId>
|
||||
<artifactId>hbase-client</artifactId>
|
||||
<version>${hbase.version}</version>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
|
||||
|
||||
<dependency>
|
||||
<groupId>org.apache.hbase</groupId>
|
||||
<artifactId>hbase-server</artifactId>
|
||||
<version>${hbase.version}</version>
|
||||
<!-- Unfortunately, HFile is packaged ONLY under hbase-server -->
|
||||
<!-- Unfortunately, HFile is packaged ONLY under hbase-server -->
|
||||
<scope>compile</scope>
|
||||
<exclusions>
|
||||
<exclusion>
|
||||
|
||||
@@ -33,10 +33,12 @@ import org.apache.hudi.common.util.ValidationUtils;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.io.storage.HoodieHFileUtils;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.CellComparatorImpl;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
@@ -178,9 +180,7 @@ public class HFileBootstrapIndex extends BootstrapIndex {
|
||||
private static HFile.Reader createReader(String hFilePath, Configuration conf, FileSystem fileSystem) {
|
||||
try {
|
||||
LOG.info("Opening HFile for reading :" + hFilePath);
|
||||
HFile.Reader reader = HFile.createReader(fileSystem, new HFilePathForReader(hFilePath),
|
||||
new CacheConfig(conf), conf);
|
||||
return reader;
|
||||
return HoodieHFileUtils.createHFileReader(fileSystem, new HFilePathForReader(hFilePath), new CacheConfig(conf), conf);
|
||||
} catch (IOException ioe) {
|
||||
throw new HoodieIOException(ioe.getMessage(), ioe);
|
||||
}
|
||||
@@ -259,7 +259,7 @@ public class HFileBootstrapIndex extends BootstrapIndex {
|
||||
|
||||
private HoodieBootstrapIndexInfo fetchBootstrapIndexInfo() throws IOException {
|
||||
return TimelineMetadataUtils.deserializeAvroMetadata(
|
||||
partitionIndexReader().loadFileInfo().get(INDEX_INFO_KEY),
|
||||
partitionIndexReader().getHFileInfo().get(INDEX_INFO_KEY),
|
||||
HoodieBootstrapIndexInfo.class);
|
||||
}
|
||||
|
||||
@@ -306,7 +306,7 @@ public class HFileBootstrapIndex extends BootstrapIndex {
|
||||
try {
|
||||
boolean available = scanner.seekTo();
|
||||
while (available) {
|
||||
keys.add(converter.apply(getUserKeyFromCellKey(CellUtil.getCellKeyAsString(scanner.getKeyValue()))));
|
||||
keys.add(converter.apply(getUserKeyFromCellKey(CellUtil.getCellKeyAsString(scanner.getCell()))));
|
||||
available = scanner.next();
|
||||
}
|
||||
} catch (IOException ioe) {
|
||||
@@ -528,13 +528,13 @@ public class HFileBootstrapIndex extends BootstrapIndex {
|
||||
@Override
|
||||
public void begin() {
|
||||
try {
|
||||
HFileContext meta = new HFileContextBuilder().build();
|
||||
HFileContext meta = new HFileContextBuilder().withCellComparator(new HoodieKVComparator()).build();
|
||||
this.indexByPartitionWriter = HFile.getWriterFactory(metaClient.getHadoopConf(),
|
||||
new CacheConfig(metaClient.getHadoopConf())).withPath(metaClient.getFs(), indexByPartitionPath)
|
||||
.withFileContext(meta).withComparator(new HoodieKVComparator()).create();
|
||||
.withFileContext(meta).create();
|
||||
this.indexByFileIdWriter = HFile.getWriterFactory(metaClient.getHadoopConf(),
|
||||
new CacheConfig(metaClient.getHadoopConf())).withPath(metaClient.getFs(), indexByFileIdPath)
|
||||
.withFileContext(meta).withComparator(new HoodieKVComparator()).create();
|
||||
.withFileContext(meta).create();
|
||||
} catch (IOException ioe) {
|
||||
throw new HoodieIOException(ioe.getMessage(), ioe);
|
||||
}
|
||||
@@ -581,6 +581,6 @@ public class HFileBootstrapIndex extends BootstrapIndex {
|
||||
* This class is explicitly used as Key Comparator to workaround hard coded
|
||||
* legacy format class names inside HBase. Otherwise we will face issues with shading.
|
||||
*/
|
||||
public static class HoodieKVComparator extends KeyValue.KVComparator {
|
||||
public static class HoodieKVComparator extends CellComparatorImpl {
|
||||
}
|
||||
}
|
||||
|
||||
@@ -424,6 +424,9 @@ public abstract class AbstractHoodieLogRecordReader {
|
||||
processDataBlock((HoodieAvroDataBlock) lastBlock, keys);
|
||||
break;
|
||||
case HFILE_DATA_BLOCK:
|
||||
if (!keys.isPresent()) {
|
||||
keys = Option.of(Collections.emptyList());
|
||||
}
|
||||
processDataBlock((HoodieHFileDataBlock) lastBlock, keys);
|
||||
break;
|
||||
case PARQUET_DATA_BLOCK:
|
||||
|
||||
@@ -208,7 +208,7 @@ public class HoodieLogFileReader implements HoodieLogFormat.Reader {
|
||||
String.format("HFile block could not be of version (%d)", HoodieLogFormatVersion.DEFAULT_VERSION));
|
||||
|
||||
return new HoodieHFileDataBlock(inputStream, content, readBlockLazily, logBlockContentLoc,
|
||||
Option.ofNullable(readerSchema), header, footer, enableRecordLookups);
|
||||
Option.ofNullable(readerSchema), header, footer, enableRecordLookups, logFile.getPath());
|
||||
|
||||
case PARQUET_DATA_BLOCK:
|
||||
checkState(nextBlockVersion.getVersion() != HoodieLogFormatVersion.DEFAULT_VERSION,
|
||||
|
||||
@@ -18,6 +18,18 @@
|
||||
|
||||
package org.apache.hudi.common.table.log.block;
|
||||
|
||||
import org.apache.hudi.avro.HoodieAvroUtils;
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.common.fs.inline.InLineFSUtils;
|
||||
import org.apache.hudi.common.fs.inline.InLineFileSystem;
|
||||
import org.apache.hudi.common.util.ClosableIterator;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.StringUtils;
|
||||
import org.apache.hudi.common.util.ValidationUtils;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.io.storage.HoodieHBaseKVComparator;
|
||||
import org.apache.hudi.io.storage.HoodieHFileReader;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
@@ -30,17 +42,6 @@ import org.apache.hadoop.hbase.io.hfile.CacheConfig;
|
||||
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.hudi.avro.HoodieAvroUtils;
|
||||
import org.apache.hudi.common.fs.inline.InLineFSUtils;
|
||||
import org.apache.hudi.common.fs.inline.InLineFileSystem;
|
||||
import org.apache.hudi.common.util.ClosableIterator;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.StringUtils;
|
||||
import org.apache.hudi.common.util.ValidationUtils;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.io.storage.HoodieHBaseKVComparator;
|
||||
import org.apache.hudi.io.storage.HoodieHFileReader;
|
||||
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
@@ -65,6 +66,9 @@ public class HoodieHFileDataBlock extends HoodieDataBlock {
|
||||
private static final int DEFAULT_BLOCK_SIZE = 1024 * 1024;
|
||||
|
||||
private final Option<Compression.Algorithm> compressionAlgorithm;
|
||||
// This path is used for constructing HFile reader context, which should not be
|
||||
// interpreted as the actual file path for the HFile data blocks
|
||||
private final Path pathForReader;
|
||||
|
||||
public HoodieHFileDataBlock(FSDataInputStream inputStream,
|
||||
Option<byte[]> content,
|
||||
@@ -73,16 +77,20 @@ public class HoodieHFileDataBlock extends HoodieDataBlock {
|
||||
Option<Schema> readerSchema,
|
||||
Map<HeaderMetadataType, String> header,
|
||||
Map<HeaderMetadataType, String> footer,
|
||||
boolean enablePointLookups) {
|
||||
boolean enablePointLookups,
|
||||
Path pathForReader) {
|
||||
super(content, inputStream, readBlockLazily, Option.of(logBlockContentLocation), readerSchema, header, footer, HoodieHFileReader.KEY_FIELD_NAME, enablePointLookups);
|
||||
this.compressionAlgorithm = Option.empty();
|
||||
this.pathForReader = pathForReader;
|
||||
}
|
||||
|
||||
public HoodieHFileDataBlock(List<IndexedRecord> records,
|
||||
Map<HeaderMetadataType, String> header,
|
||||
Compression.Algorithm compressionAlgorithm) {
|
||||
Compression.Algorithm compressionAlgorithm,
|
||||
Path pathForReader) {
|
||||
super(records, header, new HashMap<>(), HoodieHFileReader.KEY_FIELD_NAME);
|
||||
this.compressionAlgorithm = Option.of(compressionAlgorithm);
|
||||
this.pathForReader = pathForReader;
|
||||
}
|
||||
|
||||
@Override
|
||||
@@ -95,6 +103,7 @@ public class HoodieHFileDataBlock extends HoodieDataBlock {
|
||||
HFileContext context = new HFileContextBuilder()
|
||||
.withBlockSize(DEFAULT_BLOCK_SIZE)
|
||||
.withCompression(compressionAlgorithm.get())
|
||||
.withCellComparator(new HoodieHBaseKVComparator())
|
||||
.build();
|
||||
|
||||
Configuration conf = new Configuration();
|
||||
@@ -128,7 +137,7 @@ public class HoodieHFileDataBlock extends HoodieDataBlock {
|
||||
}
|
||||
|
||||
HFile.Writer writer = HFile.getWriterFactory(conf, cacheConfig)
|
||||
.withOutputStream(ostream).withFileContext(context).withComparator(new HoodieHBaseKVComparator()).create();
|
||||
.withOutputStream(ostream).withFileContext(context).create();
|
||||
|
||||
// Write the records
|
||||
sortedRecordsMap.forEach((recordKey, recordBytes) -> {
|
||||
@@ -155,7 +164,8 @@ public class HoodieHFileDataBlock extends HoodieDataBlock {
|
||||
Schema writerSchema = new Schema.Parser().parse(super.getLogBlockHeader().get(HeaderMetadataType.SCHEMA));
|
||||
|
||||
// Read the content
|
||||
HoodieHFileReader<IndexedRecord> reader = new HoodieHFileReader<>(content);
|
||||
HoodieHFileReader<IndexedRecord> reader = new HoodieHFileReader<>(
|
||||
FSUtils.getFs(pathForReader.toString(), new Configuration()), pathForReader, content);
|
||||
// Sets up the writer schema
|
||||
reader.withSchema(writerSchema);
|
||||
Iterator<IndexedRecord> recordIterator = reader.getRecordIterator(readerSchema);
|
||||
|
||||
@@ -19,11 +19,11 @@
|
||||
|
||||
package org.apache.hudi.io.storage;
|
||||
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.CellComparatorImpl;
|
||||
|
||||
/**
|
||||
* 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 {
|
||||
public class HoodieHBaseKVComparator extends CellComparatorImpl {
|
||||
}
|
||||
|
||||
@@ -18,18 +18,16 @@
|
||||
|
||||
package org.apache.hudi.io.storage;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashMap;
|
||||
import java.util.Iterator;
|
||||
import java.util.LinkedList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.TreeSet;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.hudi.avro.HoodieAvroUtils;
|
||||
import org.apache.hudi.common.bloom.BloomFilter;
|
||||
import org.apache.hudi.common.bloom.BloomFilterFactory;
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.common.util.ClosableIterator;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.ValidationUtils;
|
||||
import org.apache.hudi.common.util.io.ByteBufferBackedInputStream;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
@@ -41,26 +39,37 @@ import org.apache.hadoop.fs.PositionedReadable;
|
||||
import org.apache.hadoop.fs.Seekable;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
|
||||
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
|
||||
import org.apache.hadoop.hbase.io.hfile.HFile;
|
||||
import org.apache.hadoop.hbase.io.hfile.HFileInfo;
|
||||
import org.apache.hadoop.hbase.io.hfile.HFileScanner;
|
||||
import org.apache.hadoop.hbase.nio.ByteBuff;
|
||||
import org.apache.hadoop.hbase.util.Pair;
|
||||
import org.apache.hudi.avro.HoodieAvroUtils;
|
||||
import org.apache.hudi.common.bloom.BloomFilter;
|
||||
import org.apache.hudi.common.bloom.BloomFilterFactory;
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.common.util.ClosableIterator;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.ValidationUtils;
|
||||
import org.apache.hudi.common.util.io.ByteBufferBackedInputStream;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashMap;
|
||||
import java.util.Iterator;
|
||||
import java.util.LinkedList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.TreeSet;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
public class HoodieHFileReader<R extends IndexedRecord> implements HoodieFileReader<R> {
|
||||
public static final String KEY_FIELD_NAME = "key";
|
||||
public static final String KEY_SCHEMA = "schema";
|
||||
public static final String KEY_BLOOM_FILTER_META_BLOCK = "bloomFilter";
|
||||
public static final String KEY_BLOOM_FILTER_TYPE_CODE = "bloomFilterTypeCode";
|
||||
public static final String KEY_MIN_RECORD = "minRecordKey";
|
||||
public static final String KEY_MAX_RECORD = "maxRecordKey";
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(HoodieHFileReader.class);
|
||||
|
||||
private Path path;
|
||||
private Configuration conf;
|
||||
private HFile.Reader reader;
|
||||
@@ -70,55 +79,35 @@ public class HoodieHFileReader<R extends IndexedRecord> implements HoodieFileRea
|
||||
// key retrieval.
|
||||
private HFileScanner keyScanner;
|
||||
|
||||
public static final String KEY_FIELD_NAME = "key";
|
||||
public static final String KEY_SCHEMA = "schema";
|
||||
public static final String KEY_BLOOM_FILTER_META_BLOCK = "bloomFilter";
|
||||
public static final String KEY_BLOOM_FILTER_TYPE_CODE = "bloomFilterTypeCode";
|
||||
public static final String KEY_MIN_RECORD = "minRecordKey";
|
||||
public static final String KEY_MAX_RECORD = "maxRecordKey";
|
||||
|
||||
public HoodieHFileReader(Configuration configuration, Path path, CacheConfig cacheConfig) throws IOException {
|
||||
this.conf = configuration;
|
||||
this.path = path;
|
||||
this.reader = HFile.createReader(FSUtils.getFs(path.toString(), configuration), path, cacheConfig, conf);
|
||||
this.reader = HoodieHFileUtils.createHFileReader(FSUtils.getFs(path.toString(), configuration), path, cacheConfig, conf);
|
||||
}
|
||||
|
||||
public HoodieHFileReader(Configuration configuration, Path path, CacheConfig cacheConfig, FileSystem fs) throws IOException {
|
||||
this.conf = configuration;
|
||||
this.path = path;
|
||||
this.fsDataInputStream = fs.open(path);
|
||||
this.reader = HFile.createReader(fs, path, cacheConfig, configuration);
|
||||
this.reader = HoodieHFileUtils.createHFileReader(fs, path, cacheConfig, configuration);
|
||||
}
|
||||
|
||||
public HoodieHFileReader(byte[] content) throws IOException {
|
||||
Configuration conf = new Configuration();
|
||||
Path path = new Path("hoodie");
|
||||
SeekableByteArrayInputStream bis = new SeekableByteArrayInputStream(content);
|
||||
FSDataInputStream fsdis = new FSDataInputStream(bis);
|
||||
this.reader = HFile.createReader(FSUtils.getFs("hoodie", conf), path, new FSDataInputStreamWrapper(fsdis),
|
||||
content.length, new CacheConfig(conf), conf);
|
||||
public HoodieHFileReader(FileSystem fs, Path dummyPath, byte[] content) throws IOException {
|
||||
this.reader = HoodieHFileUtils.createHFileReader(fs, dummyPath, content);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String[] readMinMaxRecordKeys() {
|
||||
try {
|
||||
Map<byte[], byte[]> fileInfo = reader.loadFileInfo();
|
||||
return new String[] { new String(fileInfo.get(KEY_MIN_RECORD.getBytes())),
|
||||
new String(fileInfo.get(KEY_MAX_RECORD.getBytes()))};
|
||||
} catch (IOException e) {
|
||||
throw new HoodieException("Could not read min/max record key out of file information block correctly from path", e);
|
||||
}
|
||||
HFileInfo fileInfo = reader.getHFileInfo();
|
||||
return new String[] {new String(fileInfo.get(KEY_MIN_RECORD.getBytes())),
|
||||
new String(fileInfo.get(KEY_MAX_RECORD.getBytes()))};
|
||||
}
|
||||
|
||||
@Override
|
||||
public Schema getSchema() {
|
||||
if (schema == null) {
|
||||
try {
|
||||
Map<byte[], byte[]> fileInfo = reader.loadFileInfo();
|
||||
schema = new Schema.Parser().parse(new String(fileInfo.get(KEY_SCHEMA.getBytes())));
|
||||
} catch (IOException e) {
|
||||
throw new HoodieException("Could not read schema of file from path", e);
|
||||
}
|
||||
HFileInfo fileInfo = reader.getHFileInfo();
|
||||
schema = new Schema.Parser().parse(new String(fileInfo.get(KEY_SCHEMA.getBytes())));
|
||||
}
|
||||
|
||||
return schema;
|
||||
@@ -133,10 +122,10 @@ public class HoodieHFileReader<R extends IndexedRecord> implements HoodieFileRea
|
||||
|
||||
@Override
|
||||
public BloomFilter readBloomFilter() {
|
||||
Map<byte[], byte[]> fileInfo;
|
||||
HFileInfo fileInfo;
|
||||
try {
|
||||
fileInfo = reader.loadFileInfo();
|
||||
ByteBuffer serializedFilter = reader.getMetaBlock(KEY_BLOOM_FILTER_META_BLOCK, false);
|
||||
fileInfo = reader.getHFileInfo();
|
||||
ByteBuff serializedFilter = reader.getMetaBlock(KEY_BLOOM_FILTER_META_BLOCK, false).getBufferWithoutHeader();
|
||||
byte[] filterBytes = new byte[serializedFilter.remaining()];
|
||||
serializedFilter.get(filterBytes); // read the bytes that were written
|
||||
return BloomFilterFactory.fromString(new String(filterBytes),
|
||||
@@ -206,7 +195,7 @@ public class HoodieHFileReader<R extends IndexedRecord> implements HoodieFileRea
|
||||
final HFileScanner scanner = reader.getScanner(false, false);
|
||||
if (scanner.seekTo()) {
|
||||
do {
|
||||
Cell c = scanner.getKeyValue();
|
||||
Cell c = scanner.getCell();
|
||||
final Pair<String, R> keyAndRecordPair = getRecordFromCell(c, writerSchema, readerSchema, keyFieldSchema);
|
||||
recordList.add(keyAndRecordPair);
|
||||
} while (scanner.next());
|
||||
@@ -250,7 +239,6 @@ public class HoodieHFileReader<R extends IndexedRecord> implements HoodieFileRea
|
||||
*/
|
||||
public List<Pair<String, R>> readRecords(List<String> keys, Schema schema) throws IOException {
|
||||
this.schema = schema;
|
||||
reader.loadFileInfo();
|
||||
List<Pair<String, R>> records = new ArrayList<>();
|
||||
for (String key: keys) {
|
||||
Option<R> value = getRecordByKey(key, schema);
|
||||
@@ -263,7 +251,6 @@ public class HoodieHFileReader<R extends IndexedRecord> implements HoodieFileRea
|
||||
|
||||
public ClosableIterator<R> getRecordIterator(List<String> keys, Schema schema) throws IOException {
|
||||
this.schema = schema;
|
||||
reader.loadFileInfo();
|
||||
Iterator<String> iterator = keys.iterator();
|
||||
return new ClosableIterator<R>() {
|
||||
private R next;
|
||||
@@ -310,7 +297,7 @@ public class HoodieHFileReader<R extends IndexedRecord> implements HoodieFileRea
|
||||
// To handle when hasNext() is called multiple times for idempotency and/or the first time
|
||||
if (this.next == null && !this.eof) {
|
||||
if (!scanner.isSeeked() && scanner.seekTo()) {
|
||||
final Pair<String, R> keyAndRecordPair = getRecordFromCell(scanner.getKeyValue(), getSchema(), readerSchema, keyFieldSchema);
|
||||
final Pair<String, R> keyAndRecordPair = getRecordFromCell(scanner.getCell(), getSchema(), readerSchema, keyFieldSchema);
|
||||
this.next = keyAndRecordPair.getSecond();
|
||||
}
|
||||
}
|
||||
@@ -331,7 +318,7 @@ public class HoodieHFileReader<R extends IndexedRecord> implements HoodieFileRea
|
||||
}
|
||||
R retVal = this.next;
|
||||
if (scanner.next()) {
|
||||
final Pair<String, R> keyAndRecordPair = getRecordFromCell(scanner.getKeyValue(), getSchema(), readerSchema, keyFieldSchema);
|
||||
final Pair<String, R> keyAndRecordPair = getRecordFromCell(scanner.getCell(), getSchema(), readerSchema, keyFieldSchema);
|
||||
this.next = keyAndRecordPair.getSecond();
|
||||
} else {
|
||||
this.next = null;
|
||||
@@ -371,7 +358,7 @@ public class HoodieHFileReader<R extends IndexedRecord> implements HoodieFileRea
|
||||
}
|
||||
|
||||
if (keyScanner.seekTo(kv) == 0) {
|
||||
Cell c = keyScanner.getKeyValue();
|
||||
Cell c = keyScanner.getCell();
|
||||
// Extract the byte value before releasing the lock since we cannot hold on to the returned cell afterwards
|
||||
value = Arrays.copyOfRange(c.getValueArray(), c.getValueOffset(), c.getValueOffset() + c.getValueLength());
|
||||
}
|
||||
|
||||
@@ -0,0 +1,87 @@
|
||||
/*
|
||||
* 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.hadoop.fs.FSDataInputStream;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
|
||||
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
|
||||
import org.apache.hadoop.hbase.io.hfile.HFile;
|
||||
import org.apache.hadoop.hbase.io.hfile.HFileInfo;
|
||||
import org.apache.hadoop.hbase.io.hfile.ReaderContext;
|
||||
import org.apache.hadoop.hbase.io.hfile.ReaderContextBuilder;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
* Util class for HFile reading and writing in Hudi
|
||||
*/
|
||||
public class HoodieHFileUtils {
|
||||
// Based on HBase 2.4.9, the primaryReplicaReader is mainly used for constructing
|
||||
// block cache key, so if we do not use block cache then it is OK to set it as any
|
||||
// value. We use true here.
|
||||
private static final boolean USE_PRIMARY_REPLICA_READER = true;
|
||||
|
||||
/**
|
||||
* Creates HFile reader for a file with default `primaryReplicaReader` as true.
|
||||
*
|
||||
* @param fs File system.
|
||||
* @param path Path to file to read.
|
||||
* @param cacheConfig Cache configuration.
|
||||
* @param configuration Configuration
|
||||
* @return HFile reader
|
||||
* @throws IOException Upon error.
|
||||
*/
|
||||
public static HFile.Reader createHFileReader(
|
||||
FileSystem fs, Path path, CacheConfig cacheConfig, Configuration configuration) throws IOException {
|
||||
return HFile.createReader(fs, path, cacheConfig, USE_PRIMARY_REPLICA_READER, configuration);
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates HFile reader for byte array with default `primaryReplicaReader` as true.
|
||||
*
|
||||
* @param fs File system.
|
||||
* @param dummyPath Dummy path to file to read.
|
||||
* @param content Content in byte array.
|
||||
* @return HFile reader
|
||||
* @throws IOException Upon error.
|
||||
*/
|
||||
public static HFile.Reader createHFileReader(
|
||||
FileSystem fs, Path dummyPath, byte[] content) throws IOException {
|
||||
Configuration conf = new Configuration();
|
||||
HoodieHFileReader.SeekableByteArrayInputStream bis = new HoodieHFileReader.SeekableByteArrayInputStream(content);
|
||||
FSDataInputStream fsdis = new FSDataInputStream(bis);
|
||||
FSDataInputStreamWrapper stream = new FSDataInputStreamWrapper(fsdis);
|
||||
ReaderContext context = new ReaderContextBuilder()
|
||||
.withFilePath(dummyPath)
|
||||
.withInputStreamWrapper(stream)
|
||||
.withFileSize(content.length)
|
||||
.withFileSystem(fs)
|
||||
.withPrimaryReplicaReader(USE_PRIMARY_REPLICA_READER)
|
||||
.withReaderType(ReaderContext.ReaderType.STREAM)
|
||||
.build();
|
||||
HFileInfo fileInfo = new HFileInfo(context, conf);
|
||||
HFile.Reader reader = HFile.createReader(context, fileInfo, new CacheConfig(conf), conf);
|
||||
fileInfo.initMetaAndIndex(reader);
|
||||
return reader;
|
||||
}
|
||||
}
|
||||
2185
hudi-common/src/main/resources/hbase-site.xml
Normal file
2185
hudi-common/src/main/resources/hbase-site.xml
Normal file
File diff suppressed because it is too large
Load Diff
@@ -19,12 +19,13 @@
|
||||
package org.apache.hudi.common.fs.inline;
|
||||
|
||||
import org.apache.hudi.common.testutils.FileSystemTestUtils;
|
||||
import org.apache.hudi.io.storage.HoodieHBaseKVComparator;
|
||||
import org.apache.hudi.io.storage.HoodieHFileUtils;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FSDataInputStream;
|
||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
|
||||
@@ -39,10 +40,12 @@ import org.junit.jupiter.api.Test;
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashSet;
|
||||
import java.util.Set;
|
||||
import java.util.UUID;
|
||||
|
||||
import static org.apache.hadoop.hbase.CellComparatorImpl.COMPARATOR;
|
||||
import static org.apache.hudi.common.testutils.FileSystemTestUtils.FILE_SCHEME;
|
||||
import static org.apache.hudi.common.testutils.FileSystemTestUtils.RANDOM;
|
||||
import static org.apache.hudi.common.testutils.FileSystemTestUtils.getPhantomFile;
|
||||
@@ -56,11 +59,12 @@ import static org.junit.jupiter.api.Assertions.assertNotEquals;
|
||||
*/
|
||||
public class TestInLineFileSystemHFileInLining {
|
||||
|
||||
private static final String LOCAL_FORMATTER = "%010d";
|
||||
private static final String VALUE_PREFIX = "value";
|
||||
private static final int MIN_BLOCK_BYTES = 1024;
|
||||
private final Configuration inMemoryConf;
|
||||
private final Configuration inlineConf;
|
||||
private final int minBlockSize = 1024;
|
||||
private static final String LOCAL_FORMATTER = "%010d";
|
||||
private int maxRows = 100 + RANDOM.nextInt(1000);
|
||||
private final int maxRows = 100 + RANDOM.nextInt(1000);
|
||||
private Path generatedPath;
|
||||
|
||||
public TestInLineFileSystemHFileInLining() {
|
||||
@@ -88,12 +92,11 @@ public class TestInLineFileSystemHFileInLining {
|
||||
CacheConfig cacheConf = new CacheConfig(inMemoryConf);
|
||||
FSDataOutputStream fout = createFSOutput(outerInMemFSPath, inMemoryConf);
|
||||
HFileContext meta = new HFileContextBuilder()
|
||||
.withBlockSize(minBlockSize)
|
||||
.withBlockSize(MIN_BLOCK_BYTES).withCellComparator(COMPARATOR)
|
||||
.build();
|
||||
HFile.Writer writer = HFile.getWriterFactory(inMemoryConf, cacheConf)
|
||||
.withOutputStream(fout)
|
||||
.withFileContext(meta)
|
||||
.withComparator(new HoodieHBaseKVComparator())
|
||||
.create();
|
||||
|
||||
writeRecords(writer);
|
||||
@@ -110,9 +113,8 @@ public class TestInLineFileSystemHFileInLining {
|
||||
InLineFileSystem inlineFileSystem = (InLineFileSystem) inlinePath.getFileSystem(inlineConf);
|
||||
FSDataInputStream fin = inlineFileSystem.open(inlinePath);
|
||||
|
||||
HFile.Reader reader = HFile.createReader(inlineFileSystem, inlinePath, cacheConf, inlineConf);
|
||||
// Load up the index.
|
||||
reader.loadFileInfo();
|
||||
HFile.Reader reader =
|
||||
HoodieHFileUtils.createHFileReader(inlineFileSystem, inlinePath, cacheConf, inlineConf);
|
||||
// Get a scanner that caches and that does not use pread.
|
||||
HFileScanner scanner = reader.getScanner(true, false);
|
||||
// Align scanner at start of the file.
|
||||
@@ -121,21 +123,24 @@ public class TestInLineFileSystemHFileInLining {
|
||||
|
||||
Set<Integer> rowIdsToSearch = getRandomValidRowIds(10);
|
||||
for (int rowId : rowIdsToSearch) {
|
||||
assertEquals(0, scanner.seekTo(KeyValue.createKeyValueFromKey(getSomeKey(rowId))),
|
||||
KeyValue keyValue = new KeyValue.KeyOnlyKeyValue(getSomeKey(rowId));
|
||||
assertEquals(0, scanner.seekTo(keyValue),
|
||||
"location lookup failed");
|
||||
// read the key and see if it matches
|
||||
ByteBuffer readKey = scanner.getKey();
|
||||
assertArrayEquals(getSomeKey(rowId), Bytes.toBytes(readKey), "seeked key does not match");
|
||||
scanner.seekTo(KeyValue.createKeyValueFromKey(getSomeKey(rowId)));
|
||||
Cell cell = scanner.getCell();
|
||||
byte[] key = Arrays.copyOfRange(cell.getRowArray(), cell.getRowOffset(), cell.getRowOffset() + cell.getRowLength());
|
||||
byte[] expectedKey = Arrays.copyOfRange(keyValue.getRowArray(), keyValue.getRowOffset(), keyValue.getRowOffset() + keyValue.getRowLength());
|
||||
assertArrayEquals(expectedKey, key, "seeked key does not match");
|
||||
scanner.seekTo(keyValue);
|
||||
ByteBuffer val1 = scanner.getValue();
|
||||
scanner.seekTo(KeyValue.createKeyValueFromKey(getSomeKey(rowId)));
|
||||
scanner.seekTo(keyValue);
|
||||
ByteBuffer val2 = scanner.getValue();
|
||||
assertArrayEquals(Bytes.toBytes(val1), Bytes.toBytes(val2));
|
||||
}
|
||||
|
||||
int[] invalidRowIds = {-4, maxRows, maxRows + 1, maxRows + 120, maxRows + 160, maxRows + 1000};
|
||||
for (int rowId : invalidRowIds) {
|
||||
assertNotEquals(0, scanner.seekTo(KeyValue.createKeyValueFromKey(getSomeKey(rowId))),
|
||||
assertNotEquals(0, scanner.seekTo(new KeyValue.KeyOnlyKeyValue(getSomeKey(rowId))),
|
||||
"location lookup should have failed");
|
||||
}
|
||||
reader.close();
|
||||
@@ -155,7 +160,7 @@ public class TestInLineFileSystemHFileInLining {
|
||||
}
|
||||
|
||||
private byte[] getSomeKey(int rowId) {
|
||||
KeyValue kv = new KeyValue(String.format(LOCAL_FORMATTER, Integer.valueOf(rowId)).getBytes(),
|
||||
KeyValue kv = new KeyValue(String.format(LOCAL_FORMATTER, rowId).getBytes(),
|
||||
Bytes.toBytes("family"), Bytes.toBytes("qual"), HConstants.LATEST_TIMESTAMP, KeyValue.Type.Put);
|
||||
return kv.getKey();
|
||||
}
|
||||
@@ -169,17 +174,15 @@ public class TestInLineFileSystemHFileInLining {
|
||||
writer.close();
|
||||
}
|
||||
|
||||
private int writeSomeRecords(HFile.Writer writer)
|
||||
private void writeSomeRecords(HFile.Writer writer)
|
||||
throws IOException {
|
||||
String value = "value";
|
||||
KeyValue kv;
|
||||
for (int i = 0; i < (maxRows); i++) {
|
||||
String key = String.format(LOCAL_FORMATTER, Integer.valueOf(i));
|
||||
String key = String.format(LOCAL_FORMATTER, i);
|
||||
kv = new KeyValue(Bytes.toBytes(key), Bytes.toBytes("family"), Bytes.toBytes("qual"),
|
||||
Bytes.toBytes(value + key));
|
||||
Bytes.toBytes(VALUE_PREFIX + key));
|
||||
writer.append(kv);
|
||||
}
|
||||
return (maxRows);
|
||||
}
|
||||
|
||||
private void readAllRecords(HFileScanner scanner) throws IOException {
|
||||
@@ -187,30 +190,31 @@ public class TestInLineFileSystemHFileInLining {
|
||||
}
|
||||
|
||||
// read the records and check
|
||||
private int readAndCheckbytes(HFileScanner scanner, int start, int n)
|
||||
private void readAndCheckbytes(HFileScanner scanner, int start, int n)
|
||||
throws IOException {
|
||||
String value = "value";
|
||||
int i = start;
|
||||
for (; i < (start + n); i++) {
|
||||
ByteBuffer key = scanner.getKey();
|
||||
ByteBuffer val = scanner.getValue();
|
||||
String keyStr = String.format(LOCAL_FORMATTER, Integer.valueOf(i));
|
||||
String valStr = value + keyStr;
|
||||
Cell cell = scanner.getCell();
|
||||
byte[] key = Arrays.copyOfRange(
|
||||
cell.getRowArray(), cell.getRowOffset(), cell.getRowOffset() + cell.getRowLength());
|
||||
byte[] val = Arrays.copyOfRange(
|
||||
cell.getValueArray(), cell.getValueOffset(), cell.getValueOffset() + cell.getValueLength());
|
||||
String keyStr = String.format(LOCAL_FORMATTER, i);
|
||||
String valStr = VALUE_PREFIX + keyStr;
|
||||
KeyValue kv = new KeyValue(Bytes.toBytes(keyStr), Bytes.toBytes("family"),
|
||||
Bytes.toBytes("qual"), Bytes.toBytes(valStr));
|
||||
byte[] keyBytes = new KeyValue.KeyOnlyKeyValue(Bytes.toBytes(key), 0,
|
||||
Bytes.toBytes(key).length).getKey();
|
||||
assertArrayEquals(kv.getKey(), keyBytes,
|
||||
"bytes for keys do not match " + keyStr + " " + Bytes.toString(Bytes.toBytes(key)));
|
||||
byte[] valBytes = Bytes.toBytes(val);
|
||||
assertArrayEquals(Bytes.toBytes(valStr), valBytes,
|
||||
"bytes for vals do not match " + valStr + " " + Bytes.toString(valBytes));
|
||||
byte[] keyBytes = new KeyValue.KeyOnlyKeyValue(key, 0, key.length).getKey();
|
||||
byte[] expectedKeyBytes = Arrays.copyOfRange(
|
||||
kv.getRowArray(), kv.getRowOffset(), kv.getRowOffset() + kv.getRowLength());
|
||||
assertArrayEquals(expectedKeyBytes, keyBytes,
|
||||
"bytes for keys do not match " + keyStr + " " + Bytes.toString(key));
|
||||
assertArrayEquals(Bytes.toBytes(valStr), val,
|
||||
"bytes for vals do not match " + valStr + " " + Bytes.toString(val));
|
||||
if (!scanner.next()) {
|
||||
break;
|
||||
}
|
||||
}
|
||||
assertEquals(i, start + n - 1);
|
||||
return (start + n);
|
||||
}
|
||||
|
||||
private long generateOuterFile(Path outerPath, byte[] inlineBytes) throws IOException {
|
||||
|
||||
@@ -1886,11 +1886,16 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
|
||||
|
||||
private HoodieDataBlock getDataBlock(HoodieLogBlockType dataBlockType, List<IndexedRecord> records,
|
||||
Map<HeaderMetadataType, String> header) {
|
||||
return getDataBlock(dataBlockType, records, header, new Path("dummy_path"));
|
||||
}
|
||||
|
||||
private HoodieDataBlock getDataBlock(HoodieLogBlockType dataBlockType, List<IndexedRecord> records,
|
||||
Map<HeaderMetadataType, String> header, Path pathForReader) {
|
||||
switch (dataBlockType) {
|
||||
case AVRO_DATA_BLOCK:
|
||||
return new HoodieAvroDataBlock(records, header, HoodieRecord.RECORD_KEY_METADATA_FIELD);
|
||||
case HFILE_DATA_BLOCK:
|
||||
return new HoodieHFileDataBlock(records, header, Compression.Algorithm.GZ);
|
||||
return new HoodieHFileDataBlock(records, header, Compression.Algorithm.GZ, pathForReader);
|
||||
case PARQUET_DATA_BLOCK:
|
||||
return new HoodieParquetDataBlock(records, header, HoodieRecord.RECORD_KEY_METADATA_FIELD, CompressionCodecName.GZIP);
|
||||
default:
|
||||
|
||||
@@ -18,9 +18,6 @@
|
||||
|
||||
package org.apache.hudi.hadoop.testutils;
|
||||
|
||||
import org.apache.hadoop.fs.LocalFileSystem;
|
||||
import org.apache.hadoop.fs.RawLocalFileSystem;
|
||||
import org.apache.hadoop.hbase.io.compress.Compression;
|
||||
import org.apache.hudi.avro.HoodieAvroUtils;
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.common.model.HoodieFileFormat;
|
||||
@@ -44,7 +41,10 @@ import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.LocalFileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.RawLocalFileSystem;
|
||||
import org.apache.hadoop.hbase.io.compress.Compression;
|
||||
import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
|
||||
import org.apache.hadoop.hive.serde2.ColumnProjectionUtils;
|
||||
import org.apache.hadoop.mapred.JobConf;
|
||||
@@ -373,7 +373,8 @@ public class InputFormatTestUtil {
|
||||
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, writeSchema.toString());
|
||||
HoodieDataBlock dataBlock = null;
|
||||
if (logBlockType == HoodieLogBlock.HoodieLogBlockType.HFILE_DATA_BLOCK) {
|
||||
dataBlock = new HoodieHFileDataBlock(records, header, Compression.Algorithm.GZ);
|
||||
dataBlock = new HoodieHFileDataBlock(
|
||||
records, header, Compression.Algorithm.GZ, writer.getLogFile().getPath());
|
||||
} else if (logBlockType == HoodieLogBlock.HoodieLogBlockType.PARQUET_DATA_BLOCK) {
|
||||
dataBlock = new HoodieParquetDataBlock(records, header, HoodieRecord.RECORD_KEY_METADATA_FIELD, CompressionCodecName.GZIP);
|
||||
} else {
|
||||
|
||||
@@ -223,7 +223,7 @@ public abstract class ITTestBase {
|
||||
|
||||
boolean completed =
|
||||
dockerClient.execStartCmd(createCmdResponse.getId()).withDetach(false).withTty(false).exec(callback)
|
||||
.awaitCompletion(540, SECONDS);
|
||||
.awaitCompletion(540, SECONDS);
|
||||
if (!completed) {
|
||||
callback.getStderr().flush();
|
||||
callback.getStdout().flush();
|
||||
|
||||
@@ -70,6 +70,7 @@
|
||||
<resource>META-INF/LICENSE</resource>
|
||||
<file>target/classes/META-INF/LICENSE</file>
|
||||
</transformer>
|
||||
<transformer implementation="org.apache.maven.plugins.shade.resource.ServicesResourceTransformer"/>
|
||||
</transformers>
|
||||
<artifactSet>
|
||||
<includes>
|
||||
@@ -138,7 +139,7 @@
|
||||
<include>org.apache.hive:hive-service</include>
|
||||
<include>org.apache.hive:hive-service-rpc</include>
|
||||
<include>org.apache.hive:hive-exec</include>
|
||||
<include>org.apache.hive:hive-standalone-metastore</include>
|
||||
<include>org.apache.hive:hive-standalone-metastore</include>
|
||||
<include>org.apache.hive:hive-metastore</include>
|
||||
<include>org.apache.hive:hive-jdbc</include>
|
||||
<include>org.datanucleus:datanucleus-core</include>
|
||||
@@ -148,10 +149,18 @@
|
||||
|
||||
<include>org.apache.hbase:hbase-common</include>
|
||||
<include>org.apache.hbase:hbase-client</include>
|
||||
<include>org.apache.hbase:hbase-hadoop-compat</include>
|
||||
<include>org.apache.hbase:hbase-hadoop2-compat</include>
|
||||
<include>org.apache.hbase:hbase-metrics</include>
|
||||
<include>org.apache.hbase:hbase-metrics-api</include>
|
||||
<include>org.apache.hbase:hbase-server</include>
|
||||
<include>org.apache.hbase:hbase-protocol</include>
|
||||
<include>org.apache.htrace:htrace-core</include>
|
||||
<include>org.apache.hbase:hbase-protocol-shaded</include>
|
||||
<include>org.apache.hbase.thirdparty:hbase-shaded-miscellaneous</include>
|
||||
<include>org.apache.hbase.thirdparty:hbase-shaded-netty</include>
|
||||
<include>org.apache.hbase.thirdparty:hbase-shaded-protobuf</include>
|
||||
<include>org.apache.htrace:htrace-core4</include>
|
||||
<include>commons-codec:commons-codec</include>
|
||||
<include>commons-io:commons-io</include>
|
||||
</includes>
|
||||
</artifactSet>
|
||||
<relocations>
|
||||
@@ -163,6 +172,25 @@
|
||||
<pattern>org.apache.avro.</pattern>
|
||||
<shadedPattern>${flink.bundle.shade.prefix}org.apache.avro.</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.commons.io.</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.commons.io.</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.hbase.</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.hbase.</shadedPattern>
|
||||
<excludes>
|
||||
<exclude>org.apache.hadoop.hbase.KeyValue$KeyComparator</exclude>
|
||||
</excludes>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hbase.</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hbase.</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.htrace.</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.htrace.</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>com.yammer.metrics.</pattern>
|
||||
<shadedPattern>${flink.bundle.shade.prefix}com.yammer.metrics.</shadedPattern>
|
||||
@@ -192,6 +220,74 @@
|
||||
<pattern>com.fasterxml.jackson.</pattern>
|
||||
<shadedPattern>${flink.bundle.shade.prefix}com.fasterxml.jackson.</shadedPattern>
|
||||
</relocation>
|
||||
<!-- The classes below in org.apache.hadoop.metrics2 package come from
|
||||
hbase-hadoop-compat and hbase-hadoop2-compat, which have to be shaded one by one,
|
||||
instead of shading all classes under org.apache.hadoop.metrics2 including ones
|
||||
from hadoop. -->
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.MetricHistogram</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.MetricHistogram
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.MetricsExecutor</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.MetricsExecutor
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.impl.JmxCacheBuster</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.impl.JmxCacheBuster</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.lib.DefaultMetricsSystemHelper</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.lib.DefaultMetricsSystemHelper
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.lib.DynamicMetricsRegistry</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.lib.DynamicMetricsRegistry
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.lib.MetricsExecutorImpl</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.lib.MetricsExecutorImpl
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.lib.MutableFastCounter</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableFastCounter
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.lib.MutableHistogram</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableHistogram
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.lib.MutableRangeHistogram</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableRangeHistogram
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.lib.MutableSizeHistogram</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableSizeHistogram
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.lib.MutableTimeHistogram</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableTimeHistogram
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.util.MetricQuantile</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.util.MetricQuantile
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.util.MetricSampleQuantiles</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.util.MetricSampleQuantiles
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
</relocations>
|
||||
<filters>
|
||||
<filter>
|
||||
@@ -201,6 +297,8 @@
|
||||
<exclude>META-INF/*.DSA</exclude>
|
||||
<exclude>META-INF/*.RSA</exclude>
|
||||
<exclude>META-INF/services/javax.*</exclude>
|
||||
<exclude>**/*.proto</exclude>
|
||||
<exclude>hbase-webapps/**</exclude>
|
||||
</excludes>
|
||||
</filter>
|
||||
</filters>
|
||||
@@ -273,11 +371,23 @@
|
||||
<groupId>org.apache.hudi</groupId>
|
||||
<artifactId>hudi-hadoop-mr</artifactId>
|
||||
<version>${project.version}</version>
|
||||
<exclusions>
|
||||
<exclusion>
|
||||
<artifactId>guava</artifactId>
|
||||
<groupId>com.google.guava</groupId>
|
||||
</exclusion>
|
||||
</exclusions>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.hudi</groupId>
|
||||
<artifactId>hudi-hive-sync</artifactId>
|
||||
<version>${project.version}</version>
|
||||
<exclusions>
|
||||
<exclusion>
|
||||
<artifactId>guava</artifactId>
|
||||
<groupId>com.google.guava</groupId>
|
||||
</exclusion>
|
||||
</exclusions>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.hudi</groupId>
|
||||
@@ -288,6 +398,10 @@
|
||||
<artifactId>rocksdbjni</artifactId>
|
||||
<groupId>org.rocksdb</groupId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<artifactId>guava</artifactId>
|
||||
<groupId>com.google.guava</groupId>
|
||||
</exclusion>
|
||||
</exclusions>
|
||||
</dependency>
|
||||
<dependency>
|
||||
@@ -551,66 +665,6 @@
|
||||
<artifactId>jackson-annotations</artifactId>
|
||||
<scope>compile</scope>
|
||||
</dependency>
|
||||
|
||||
<!-- Hbase -->
|
||||
<dependency>
|
||||
<groupId>org.apache.hbase</groupId>
|
||||
<artifactId>hbase-common</artifactId>
|
||||
<version>${hbase.version}</version>
|
||||
<exclusions>
|
||||
<exclusion>
|
||||
<artifactId>guava</artifactId>
|
||||
<groupId>com.google.guava</groupId>
|
||||
</exclusion>
|
||||
</exclusions>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.hbase</groupId>
|
||||
<artifactId>hbase-server</artifactId>
|
||||
<version>${hbase.version}</version>
|
||||
<scope>compile</scope>
|
||||
<exclusions>
|
||||
<exclusion>
|
||||
<artifactId>guava</artifactId>
|
||||
<groupId>com.google.guava</groupId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<groupId>org.apache.hbase</groupId>
|
||||
<artifactId>hbase-common</artifactId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<groupId>javax.servlet</groupId>
|
||||
<artifactId>*</artifactId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<groupId>org.codehaus.jackson</groupId>
|
||||
<artifactId>*</artifactId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<groupId>org.mortbay.jetty</groupId>
|
||||
<artifactId>*</artifactId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<groupId>tomcat</groupId>
|
||||
<artifactId>*</artifactId>
|
||||
</exclusion>
|
||||
</exclusions>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.hbase</groupId>
|
||||
<artifactId>hbase-client</artifactId>
|
||||
<version>${hbase.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.hbase</groupId>
|
||||
<artifactId>hbase-protocol</artifactId>
|
||||
<version>${hbase.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.htrace</groupId>
|
||||
<artifactId>htrace-core</artifactId>
|
||||
<version>${htrace.version}</version>
|
||||
</dependency>
|
||||
</dependencies>
|
||||
|
||||
<profiles>
|
||||
|
||||
@@ -55,12 +55,13 @@
|
||||
<transformer implementation="org.apache.maven.plugins.shade.resource.ApacheLicenseResourceTransformer">
|
||||
</transformer>
|
||||
<transformer implementation="org.apache.maven.plugins.shade.resource.ApacheNoticeResourceTransformer">
|
||||
<addHeader>true</addHeader>
|
||||
<addHeader>true</addHeader>
|
||||
</transformer>
|
||||
<transformer implementation="org.apache.maven.plugins.shade.resource.IncludeResourceTransformer">
|
||||
<resource>META-INF/LICENSE</resource>
|
||||
<file>target/classes/META-INF/LICENSE</file>
|
||||
</transformer>
|
||||
<transformer implementation="org.apache.maven.plugins.shade.resource.ServicesResourceTransformer"/>
|
||||
</transformers>
|
||||
<artifactSet>
|
||||
<includes>
|
||||
@@ -74,11 +75,19 @@
|
||||
<include>com.esotericsoftware:minlog</include>
|
||||
<include>org.apache.hbase:hbase-common</include>
|
||||
<include>org.apache.hbase:hbase-client</include>
|
||||
<include>org.apache.hbase:hbase-protocol</include>
|
||||
<include>org.apache.hbase:hbase-hadoop-compat</include>
|
||||
<include>org.apache.hbase:hbase-hadoop2-compat</include>
|
||||
<include>org.apache.hbase:hbase-metrics</include>
|
||||
<include>org.apache.hbase:hbase-metrics-api</include>
|
||||
<include>org.apache.hbase:hbase-protocol-shaded</include>
|
||||
<include>org.apache.hbase:hbase-server</include>
|
||||
<include>org.apache.htrace:htrace-core</include>
|
||||
<include>org.apache.hbase.thirdparty:hbase-shaded-miscellaneous</include>
|
||||
<include>org.apache.hbase.thirdparty:hbase-shaded-netty</include>
|
||||
<include>org.apache.hbase.thirdparty:hbase-shaded-protobuf</include>
|
||||
<include>org.apache.htrace:htrace-core4</include>
|
||||
<include>com.yammer.metrics:metrics-core</include>
|
||||
<include>com.google.guava:guava</include>
|
||||
<include>commons-io:commons-io</include>
|
||||
</includes>
|
||||
</artifactSet>
|
||||
<relocations>
|
||||
@@ -102,6 +111,25 @@
|
||||
<pattern>org.apache.avro.</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.avro.</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.commons.io.</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.commons.io.</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.hbase.</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.hbase.</shadedPattern>
|
||||
<excludes>
|
||||
<exclude>org.apache.hadoop.hbase.KeyValue$KeyComparator</exclude>
|
||||
</excludes>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hbase.</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hbase.</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.htrace.</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.htrace.</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.parquet.avro.</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.parquet.avro.</shadedPattern>
|
||||
@@ -110,6 +138,74 @@
|
||||
<pattern>com.google.common.</pattern>
|
||||
<shadedPattern>org.apache.hudi.com.google.common.</shadedPattern>
|
||||
</relocation>
|
||||
<!-- The classes below in org.apache.hadoop.metrics2 package come from
|
||||
hbase-hadoop-compat and hbase-hadoop2-compat, which have to be shaded one by one,
|
||||
instead of shading all classes under org.apache.hadoop.metrics2 including ones
|
||||
from hadoop. -->
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.MetricHistogram</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.MetricHistogram
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.MetricsExecutor</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.MetricsExecutor
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.impl.JmxCacheBuster</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.impl.JmxCacheBuster</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.lib.DefaultMetricsSystemHelper</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.lib.DefaultMetricsSystemHelper
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.lib.DynamicMetricsRegistry</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.lib.DynamicMetricsRegistry
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.lib.MetricsExecutorImpl</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.lib.MetricsExecutorImpl
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.lib.MutableFastCounter</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableFastCounter
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.lib.MutableHistogram</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableHistogram
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.lib.MutableRangeHistogram</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableRangeHistogram
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.lib.MutableSizeHistogram</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableSizeHistogram
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.lib.MutableTimeHistogram</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableTimeHistogram
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.util.MetricQuantile</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.util.MetricQuantile
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.util.MetricSampleQuantiles</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.util.MetricSampleQuantiles
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
</relocations>
|
||||
<createDependencyReducedPom>false</createDependencyReducedPom>
|
||||
<filters>
|
||||
@@ -120,6 +216,8 @@
|
||||
<exclude>META-INF/*.DSA</exclude>
|
||||
<exclude>META-INF/*.RSA</exclude>
|
||||
<exclude>META-INF/services/javax.*</exclude>
|
||||
<exclude>**/*.proto</exclude>
|
||||
<exclude>hbase-webapps/**</exclude>
|
||||
</excludes>
|
||||
</filter>
|
||||
</filters>
|
||||
@@ -167,48 +265,5 @@
|
||||
<version>${avro.version}</version>
|
||||
<scope>compile</scope>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>org.apache.htrace</groupId>
|
||||
<artifactId>htrace-core</artifactId>
|
||||
<version>${htrace.version}</version>
|
||||
<scope>compile</scope>
|
||||
</dependency>
|
||||
|
||||
<!-- HBase -->
|
||||
<dependency>
|
||||
<groupId>org.apache.hbase</groupId>
|
||||
<artifactId>hbase-common</artifactId>
|
||||
<version>${hbase.version}</version>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>org.apache.hbase</groupId>
|
||||
<artifactId>hbase-server</artifactId>
|
||||
<version>${hbase.version}</version>
|
||||
<scope>compile</scope>
|
||||
<exclusions>
|
||||
<exclusion>
|
||||
<groupId>org.apache.hbase</groupId>
|
||||
<artifactId>hbase-common</artifactId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<groupId>javax.servlet</groupId>
|
||||
<artifactId>*</artifactId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<groupId>org.codehaus.jackson</groupId>
|
||||
<artifactId>*</artifactId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<groupId>org.mortbay.jetty</groupId>
|
||||
<artifactId>*</artifactId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<groupId>tomcat</groupId>
|
||||
<artifactId>*</artifactId>
|
||||
</exclusion>
|
||||
</exclusions>
|
||||
</dependency>
|
||||
</dependencies>
|
||||
</project>
|
||||
|
||||
@@ -62,6 +62,7 @@
|
||||
<transformer implementation="org.apache.maven.plugins.shade.resource.AppendingTransformer">
|
||||
<resource>META-INF/services/org.apache.spark.sql.sources.DataSourceRegister</resource>
|
||||
</transformer>
|
||||
<transformer implementation="org.apache.maven.plugins.shade.resource.ServicesResourceTransformer"/>
|
||||
</transformers>
|
||||
<artifactSet>
|
||||
<includes>
|
||||
@@ -85,6 +86,20 @@
|
||||
<include>org.apache.hudi:hudi-aws</include>
|
||||
<include>org.apache.hudi:hudi-integ-test</include>
|
||||
|
||||
<include>org.apache.hbase:hbase-common</include>
|
||||
<include>org.apache.hbase:hbase-client</include>
|
||||
<include>org.apache.hbase:hbase-hadoop-compat</include>
|
||||
<include>org.apache.hbase:hbase-hadoop2-compat</include>
|
||||
<include>org.apache.hbase:hbase-metrics</include>
|
||||
<include>org.apache.hbase:hbase-metrics-api</include>
|
||||
<include>org.apache.hbase:hbase-protocol-shaded</include>
|
||||
<include>org.apache.hbase:hbase-server</include>
|
||||
<include>org.apache.hbase.thirdparty:hbase-shaded-miscellaneous</include>
|
||||
<include>org.apache.hbase.thirdparty:hbase-shaded-netty</include>
|
||||
<include>org.apache.hbase.thirdparty:hbase-shaded-protobuf</include>
|
||||
<include>org.apache.htrace:htrace-core4</include>
|
||||
<include>commons-io:commons-io</include>
|
||||
|
||||
<include>org.jetbrains.kotlin:kotlin-stdlib-jdk8</include>
|
||||
<include>org.jetbrains.kotlin:kotlin-stdlib</include>
|
||||
<include>org.jetbrains.kotlin:kotlin-stdlib-common</include>
|
||||
@@ -133,7 +148,6 @@
|
||||
|
||||
<include>org.apache.hive:hive-common</include>
|
||||
<include>org.apache.hive:hive-service</include>
|
||||
<include>org.apache.hive:hive-metastore</include>
|
||||
<include>org.apache.hive:hive-jdbc</include>
|
||||
<include>org.apache.hive:hive-exec</include>
|
||||
|
||||
@@ -156,7 +170,6 @@
|
||||
<include>com.fasterxml.jackson.core:jackson-databind</include>
|
||||
<include>com.fasterxml.jackson.dataformat:jackson-dataformat-yaml</include>
|
||||
|
||||
<include>org.apache.htrace:htrace-core</include>
|
||||
<include>org.apache.curator:curator-framework</include>
|
||||
<include>org.apache.curator:curator-client</include>
|
||||
<include>org.apache.curator:curator-recipes</include>
|
||||
@@ -179,6 +192,25 @@
|
||||
<pattern>org.apache.commons.pool.</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.commons.pool.</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.commons.io.</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.commons.io.</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.hbase.</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.hbase.</shadedPattern>
|
||||
<excludes>
|
||||
<exclude>org.apache.hadoop.hbase.KeyValue$KeyComparator</exclude>
|
||||
</excludes>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hbase.</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hbase.</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.htrace.</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.htrace.</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hive.jdbc.</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hive.jdbc.</shadedPattern>
|
||||
@@ -259,6 +291,74 @@
|
||||
<pattern>org.apache.parquet.avro.</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.parquet.avro.</shadedPattern>
|
||||
</relocation>
|
||||
<!-- The classes below in org.apache.hadoop.metrics2 package come from
|
||||
hbase-hadoop-compat and hbase-hadoop2-compat, which have to be shaded one by one,
|
||||
instead of shading all classes under org.apache.hadoop.metrics2 including ones
|
||||
from hadoop. -->
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.MetricHistogram</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.MetricHistogram
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.MetricsExecutor</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.MetricsExecutor
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.impl.JmxCacheBuster</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.impl.JmxCacheBuster</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.lib.DefaultMetricsSystemHelper</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.lib.DefaultMetricsSystemHelper
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.lib.DynamicMetricsRegistry</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.lib.DynamicMetricsRegistry
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.lib.MetricsExecutorImpl</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.lib.MetricsExecutorImpl
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.lib.MutableFastCounter</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableFastCounter
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.lib.MutableHistogram</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableHistogram
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.lib.MutableRangeHistogram</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableRangeHistogram
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.lib.MutableSizeHistogram</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableSizeHistogram
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.lib.MutableTimeHistogram</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableTimeHistogram
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.util.MetricQuantile</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.util.MetricQuantile
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.util.MetricSampleQuantiles</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.util.MetricSampleQuantiles
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
</relocations>
|
||||
<filters>
|
||||
<filter>
|
||||
@@ -270,6 +370,8 @@
|
||||
<!-- Use this jar's NOTICE and license file -->
|
||||
<exclude>META-INF/NOTICE*</exclude>
|
||||
<exclude>META-INF/LICENSE*</exclude>
|
||||
<exclude>**/*.proto</exclude>
|
||||
<exclude>hbase-webapps/**</exclude>
|
||||
</excludes>
|
||||
</filter>
|
||||
</filters>
|
||||
@@ -398,6 +500,12 @@
|
||||
<artifactId>hive-metastore</artifactId>
|
||||
<version>${hive.version}</version>
|
||||
<scope>provided</scope>
|
||||
<exclusions>
|
||||
<exclusion>
|
||||
<groupId>org.apache.hbase</groupId>
|
||||
<artifactId>*</artifactId>
|
||||
</exclusion>
|
||||
</exclusions>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
|
||||
@@ -58,14 +58,16 @@
|
||||
implementation="org.apache.maven.plugins.shade.resource.ApacheLicenseResourceTransformer">
|
||||
</transformer>
|
||||
<transformer
|
||||
implementation="org.apache.maven.plugins.shade.resource.ApacheNoticeResourceTransformer">
|
||||
implementation="org.apache.maven.plugins.shade.resource.ApacheNoticeResourceTransformer">
|
||||
<addHeader>true</addHeader>
|
||||
</transformer>
|
||||
<transformer
|
||||
implementation="org.apache.maven.plugins.shade.resource.IncludeResourceTransformer">
|
||||
implementation="org.apache.maven.plugins.shade.resource.IncludeResourceTransformer">
|
||||
<resource>META-INF/LICENSE</resource>
|
||||
<file>target/classes/META-INF/LICENSE</file>
|
||||
</transformer>
|
||||
<transformer
|
||||
implementation="org.apache.maven.plugins.shade.resource.ServicesResourceTransformer"/>
|
||||
</transformers>
|
||||
<artifactSet>
|
||||
<includes>
|
||||
@@ -115,13 +117,21 @@
|
||||
<include>org.objenesis:objenesis</include>
|
||||
<include>com.esotericsoftware:kryo-shaded</include>
|
||||
<include>com.esotericsoftware:minlog</include>
|
||||
|
||||
|
||||
<include>org.apache.hbase:hbase-client</include>
|
||||
<include>org.apache.hbase:hbase-common</include>
|
||||
<include>org.apache.hbase:hbase-protocol</include>
|
||||
<include>org.apache.hbase:hbase-hadoop-compat</include>
|
||||
<include>org.apache.hbase:hbase-hadoop2-compat</include>
|
||||
<include>org.apache.hbase:hbase-metrics</include>
|
||||
<include>org.apache.hbase:hbase-metrics-api</include>
|
||||
<include>org.apache.hbase:hbase-protocol-shaded</include>
|
||||
<include>org.apache.hbase:hbase-server</include>
|
||||
<include>org.apache.htrace:htrace-core</include>
|
||||
<include>org.apache.hbase.thirdparty:hbase-shaded-miscellaneous</include>
|
||||
<include>org.apache.hbase.thirdparty:hbase-shaded-netty</include>
|
||||
<include>org.apache.hbase.thirdparty:hbase-shaded-protobuf</include>
|
||||
<include>org.apache.htrace:htrace-core4</include>
|
||||
<include>org.scala-lang:*</include>
|
||||
<include>commons-io:commons-io</include>
|
||||
</includes>
|
||||
</artifactSet>
|
||||
<relocations>
|
||||
@@ -131,15 +141,107 @@
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>com.yammer.metrics.</pattern>
|
||||
<shadedPattern>${kafka.connect.bundle.shade.prefix}com.yammer.metrics.</shadedPattern>
|
||||
<shadedPattern>${kafka.connect.bundle.shade.prefix}com.yammer.metrics.
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>com.beust.jcommander.</pattern>
|
||||
<shadedPattern>${kafka.connect.bundle.shade.prefix}com.beust.jcommander.</shadedPattern>
|
||||
<shadedPattern>${kafka.connect.bundle.shade.prefix}com.beust.jcommander.
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.eclipse.jetty.</pattern>
|
||||
<shadedPattern>${kafka.connect.bundle.shade.prefix}org.eclipse.jetty.</shadedPattern>
|
||||
<shadedPattern>${kafka.connect.bundle.shade.prefix}org.eclipse.jetty.
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.commons.io.</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.commons.io.</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.hbase.</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.hbase.</shadedPattern>
|
||||
<excludes>
|
||||
<exclude>org.apache.hadoop.hbase.KeyValue$KeyComparator</exclude>
|
||||
</excludes>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hbase.</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hbase.</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.htrace.</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.htrace.</shadedPattern>
|
||||
</relocation>
|
||||
<!-- The classes below in org.apache.hadoop.metrics2 package come from
|
||||
hbase-hadoop-compat and hbase-hadoop2-compat, which have to be shaded one by one,
|
||||
instead of shading all classes under org.apache.hadoop.metrics2 including ones
|
||||
from hadoop. -->
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.MetricHistogram</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.MetricHistogram
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.MetricsExecutor</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.MetricsExecutor
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.impl.JmxCacheBuster</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.impl.JmxCacheBuster
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.lib.DefaultMetricsSystemHelper</pattern>
|
||||
<shadedPattern>
|
||||
org.apache.hudi.org.apache.hadoop.metrics2.lib.DefaultMetricsSystemHelper
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.lib.DynamicMetricsRegistry</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.lib.DynamicMetricsRegistry
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.lib.MetricsExecutorImpl</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.lib.MetricsExecutorImpl
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.lib.MutableFastCounter</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableFastCounter
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.lib.MutableHistogram</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableHistogram
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.lib.MutableRangeHistogram</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableRangeHistogram
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.lib.MutableSizeHistogram</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableSizeHistogram
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.lib.MutableTimeHistogram</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableTimeHistogram
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.util.MetricQuantile</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.util.MetricQuantile
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.util.MetricSampleQuantiles</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.util.MetricSampleQuantiles
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
</relocations>
|
||||
<filters>
|
||||
@@ -150,6 +252,8 @@
|
||||
<exclude>META-INF/*.DSA</exclude>
|
||||
<exclude>META-INF/*.RSA</exclude>
|
||||
<exclude>META-INF/services/javax.*</exclude>
|
||||
<exclude>**/*.proto</exclude>
|
||||
<exclude>hbase-webapps/**</exclude>
|
||||
</excludes>
|
||||
</filter>
|
||||
</filters>
|
||||
@@ -322,13 +426,6 @@
|
||||
<scope>${utilities.bundle.hive.scope}</scope>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>org.apache.htrace</groupId>
|
||||
<artifactId>htrace-core</artifactId>
|
||||
<version>${htrace.version}</version>
|
||||
<scope>compile</scope>
|
||||
</dependency>
|
||||
|
||||
</dependencies>
|
||||
</project>
|
||||
|
||||
|
||||
@@ -61,6 +61,7 @@
|
||||
<resource>META-INF/LICENSE</resource>
|
||||
<file>target/classes/META-INF/LICENSE</file>
|
||||
</transformer>
|
||||
<transformer implementation="org.apache.maven.plugins.shade.resource.ServicesResourceTransformer"/>
|
||||
</transformers>
|
||||
<artifactSet>
|
||||
<includes>
|
||||
@@ -75,20 +76,51 @@
|
||||
<include>com.esotericsoftware:minlog</include>
|
||||
<include>org.apache.hbase:hbase-common</include>
|
||||
<include>org.apache.hbase:hbase-client</include>
|
||||
<include>org.apache.hbase:hbase-hadoop-compat</include>
|
||||
<include>org.apache.hbase:hbase-hadoop2-compat</include>
|
||||
<include>org.apache.hbase:hbase-metrics</include>
|
||||
<include>org.apache.hbase:hbase-metrics-api</include>
|
||||
<include>org.apache.hbase:hbase-protocol</include>
|
||||
<include>org.apache.hbase:hbase-server</include>
|
||||
<include>org.apache.htrace:htrace-core</include>
|
||||
<include>org.apache.hbase:hbase-protocol-shaded</include>
|
||||
<include>org.apache.hbase.thirdparty:hbase-shaded-miscellaneous</include>
|
||||
<include>org.apache.hbase.thirdparty:hbase-shaded-netty</include>
|
||||
<include>org.apache.hbase.thirdparty:hbase-shaded-protobuf</include>
|
||||
<include>org.apache.htrace:htrace-core4</include>
|
||||
<include>com.yammer.metrics:metrics-core</include>
|
||||
<include>com.google.guava:guava</include>
|
||||
<include>commons-io:commons-io</include>
|
||||
<include>commons-lang:commons-lang</include>
|
||||
<include>com.google.protobuf:protobuf-java</include>
|
||||
</includes>
|
||||
</artifactSet>
|
||||
<relocations>
|
||||
<relocation>
|
||||
<pattern>org.apache.parquet.avro.</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.parquet.avro.</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.avro.</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.avro.</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.commons.io.</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.commons.io.</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.hbase.</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.hbase.</shadedPattern>
|
||||
<excludes>
|
||||
<exclude>org.apache.hadoop.hbase.KeyValue$KeyComparator</exclude>
|
||||
</excludes>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hbase.</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hbase.</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.htrace.</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.htrace.</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.codehaus.jackson.</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.codehaus.jackson.</shadedPattern>
|
||||
@@ -121,14 +153,78 @@
|
||||
<pattern>com.google.protobuf.</pattern>
|
||||
<shadedPattern>${presto.bundle.bootstrap.shade.prefix}com.google.protobuf.</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.htrace.</pattern>
|
||||
<shadedPattern>${presto.bundle.bootstrap.shade.prefix}org.apache.htrace.</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.parquet.avro.</pattern>
|
||||
<shadedPattern>${presto.bundle.bootstrap.shade.prefix}org.apache.parquet.avro.</shadedPattern>
|
||||
</relocation>
|
||||
<!-- The classes below in org.apache.hadoop.metrics2 package come from
|
||||
hbase-hadoop-compat and hbase-hadoop2-compat, which have to be shaded one by one,
|
||||
instead of shading all classes under org.apache.hadoop.metrics2 including ones
|
||||
from hadoop. -->
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.MetricHistogram</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.MetricHistogram
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.MetricsExecutor</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.MetricsExecutor
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.impl.JmxCacheBuster</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.impl.JmxCacheBuster</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.lib.DefaultMetricsSystemHelper</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.lib.DefaultMetricsSystemHelper
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.lib.DynamicMetricsRegistry</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.lib.DynamicMetricsRegistry
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.lib.MetricsExecutorImpl</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.lib.MetricsExecutorImpl
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.lib.MutableFastCounter</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableFastCounter
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.lib.MutableHistogram</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableHistogram
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.lib.MutableRangeHistogram</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableRangeHistogram
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.lib.MutableSizeHistogram</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableSizeHistogram
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.lib.MutableTimeHistogram</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableTimeHistogram
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.util.MetricQuantile</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.util.MetricQuantile
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.util.MetricSampleQuantiles</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.util.MetricSampleQuantiles
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
</relocations>
|
||||
<createDependencyReducedPom>false</createDependencyReducedPom>
|
||||
<filters>
|
||||
@@ -139,7 +235,9 @@
|
||||
<exclude>META-INF/*.DSA</exclude>
|
||||
<exclude>META-INF/*.RSA</exclude>
|
||||
<exclude>META-INF/services/javax.*</exclude>
|
||||
<exclude>**/*.proto</exclude>
|
||||
<exclude>com/esotericsoftware/reflectasm/**</exclude>
|
||||
<exclude>hbase-webapps/**</exclude>
|
||||
<exclude>stringBehavior.avsc</exclude>
|
||||
</excludes>
|
||||
</filter>
|
||||
@@ -171,20 +269,6 @@
|
||||
<groupId>org.apache.hudi</groupId>
|
||||
<artifactId>hudi-hadoop-mr-bundle</artifactId>
|
||||
<version>${project.version}</version>
|
||||
<exclusions>
|
||||
<exclusion>
|
||||
<groupId>org.apache.hbase</groupId>
|
||||
<artifactId>hbase-common</artifactId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<groupId>org.apache.hbase</groupId>
|
||||
<artifactId>hbase-server</artifactId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<groupId>org.apache.hbase</groupId>
|
||||
<artifactId>hbase-client</artifactId>
|
||||
</exclusion>
|
||||
</exclusions>
|
||||
</dependency>
|
||||
|
||||
<!-- Parquet -->
|
||||
@@ -201,42 +285,6 @@
|
||||
<scope>compile</scope>
|
||||
</dependency>
|
||||
|
||||
<!-- HBase -->
|
||||
<dependency>
|
||||
<groupId>org.apache.hbase</groupId>
|
||||
<artifactId>hbase-common</artifactId>
|
||||
<version>${hbase.version}</version>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>org.apache.hbase</groupId>
|
||||
<artifactId>hbase-server</artifactId>
|
||||
<version>${hbase.version}</version>
|
||||
<scope>compile</scope>
|
||||
<exclusions>
|
||||
<exclusion>
|
||||
<groupId>org.apache.hbase</groupId>
|
||||
<artifactId>hbase-common</artifactId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<groupId>javax.servlet</groupId>
|
||||
<artifactId>*</artifactId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<groupId>org.codehaus.jackson</groupId>
|
||||
<artifactId>*</artifactId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<groupId>org.mortbay.jetty</groupId>
|
||||
<artifactId>*</artifactId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<groupId>tomcat</groupId>
|
||||
<artifactId>*</artifactId>
|
||||
</exclusion>
|
||||
</exclusions>
|
||||
</dependency>
|
||||
|
||||
<!--Guava needs to be shaded because HBase 1.2.3 depends on an earlier guava version i.e 12.0.1 and hits runtime
|
||||
issues with the guava version present in Presto runtime-->
|
||||
<dependency>
|
||||
|
||||
@@ -63,6 +63,7 @@
|
||||
<transformer implementation="org.apache.maven.plugins.shade.resource.AppendingTransformer">
|
||||
<resource>META-INF/services/org.apache.spark.sql.sources.DataSourceRegister</resource>
|
||||
</transformer>
|
||||
<transformer implementation="org.apache.maven.plugins.shade.resource.ServicesResourceTransformer"/>
|
||||
</transformers>
|
||||
<artifactSet>
|
||||
<includes>
|
||||
@@ -116,13 +117,21 @@
|
||||
|
||||
<include>org.apache.hbase:hbase-client</include>
|
||||
<include>org.apache.hbase:hbase-common</include>
|
||||
<include>org.apache.hbase:hbase-protocol</include>
|
||||
<include>org.apache.hbase:hbase-hadoop-compat</include>
|
||||
<include>org.apache.hbase:hbase-hadoop2-compat</include>
|
||||
<include>org.apache.hbase:hbase-metrics</include>
|
||||
<include>org.apache.hbase:hbase-metrics-api</include>
|
||||
<include>org.apache.hbase:hbase-protocol-shaded</include>
|
||||
<include>org.apache.hbase:hbase-server</include>
|
||||
<include>org.apache.htrace:htrace-core</include>
|
||||
<include>org.apache.hbase.thirdparty:hbase-shaded-miscellaneous</include>
|
||||
<include>org.apache.hbase.thirdparty:hbase-shaded-netty</include>
|
||||
<include>org.apache.hbase.thirdparty:hbase-shaded-protobuf</include>
|
||||
<include>org.apache.htrace:htrace-core4</include>
|
||||
<include>org.apache.curator:curator-framework</include>
|
||||
<include>org.apache.curator:curator-client</include>
|
||||
<include>org.apache.curator:curator-recipes</include>
|
||||
<include>commons-codec:commons-codec</include>
|
||||
<include>commons-io:commons-io</include>
|
||||
</includes>
|
||||
</artifactSet>
|
||||
<relocations>
|
||||
@@ -134,6 +143,25 @@
|
||||
<pattern>com.beust.jcommander.</pattern>
|
||||
<shadedPattern>org.apache.hudi.com.beust.jcommander.</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.commons.io.</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.commons.io.</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.hbase.</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.hbase.</shadedPattern>
|
||||
<excludes>
|
||||
<exclude>org.apache.hadoop.hbase.KeyValue$KeyComparator</exclude>
|
||||
</excludes>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hbase.</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hbase.</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.htrace.</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.htrace.</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.spark.sql.avro.</pattern>
|
||||
<shadedPattern>${spark.bundle.spark.shade.prefix}org.apache.spark.sql.avro.</shadedPattern>
|
||||
@@ -183,6 +211,74 @@
|
||||
<shadedPattern>${spark.bundle.spark.shade.prefix}com.google.common.</shadedPattern>
|
||||
</relocation>
|
||||
<!-- TODO: Revisit GH ISSUE #533 & PR#633-->
|
||||
<!-- The classes below in org.apache.hadoop.metrics2 package come from
|
||||
hbase-hadoop-compat and hbase-hadoop2-compat, which have to be shaded one by one,
|
||||
instead of shading all classes under org.apache.hadoop.metrics2 including ones
|
||||
from hadoop. -->
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.MetricHistogram</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.MetricHistogram
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.MetricsExecutor</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.MetricsExecutor
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.impl.JmxCacheBuster</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.impl.JmxCacheBuster</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.lib.DefaultMetricsSystemHelper</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.lib.DefaultMetricsSystemHelper
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.lib.DynamicMetricsRegistry</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.lib.DynamicMetricsRegistry
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.lib.MetricsExecutorImpl</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.lib.MetricsExecutorImpl
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.lib.MutableFastCounter</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableFastCounter
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.lib.MutableHistogram</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableHistogram
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.lib.MutableRangeHistogram</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableRangeHistogram
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.lib.MutableSizeHistogram</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableSizeHistogram
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.lib.MutableTimeHistogram</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableTimeHistogram
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.util.MetricQuantile</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.util.MetricQuantile
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.util.MetricSampleQuantiles</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.util.MetricSampleQuantiles
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
</relocations>
|
||||
<filters>
|
||||
<filter>
|
||||
@@ -192,6 +288,8 @@
|
||||
<exclude>META-INF/*.DSA</exclude>
|
||||
<exclude>META-INF/*.RSA</exclude>
|
||||
<exclude>META-INF/services/javax.*</exclude>
|
||||
<exclude>**/*.proto</exclude>
|
||||
<exclude>hbase-webapps/**</exclude>
|
||||
</excludes>
|
||||
</filter>
|
||||
</filters>
|
||||
@@ -314,58 +412,6 @@
|
||||
<scope>${spark.bundle.hive.scope}</scope>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>org.apache.htrace</groupId>
|
||||
<artifactId>htrace-core</artifactId>
|
||||
<version>${htrace.version}</version>
|
||||
<scope>compile</scope>
|
||||
</dependency>
|
||||
|
||||
<!-- Hbase -->
|
||||
<dependency>
|
||||
<groupId>org.apache.hbase</groupId>
|
||||
<artifactId>hbase-common</artifactId>
|
||||
<version>${hbase.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.hbase</groupId>
|
||||
<artifactId>hbase-server</artifactId>
|
||||
<version>${hbase.version}</version>
|
||||
<scope>compile</scope>
|
||||
<exclusions>
|
||||
<exclusion>
|
||||
<groupId>org.apache.hbase</groupId>
|
||||
<artifactId>hbase-common</artifactId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<groupId>javax.servlet</groupId>
|
||||
<artifactId>*</artifactId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<groupId>org.codehaus.jackson</groupId>
|
||||
<artifactId>*</artifactId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<groupId>org.mortbay.jetty</groupId>
|
||||
<artifactId>*</artifactId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<groupId>tomcat</groupId>
|
||||
<artifactId>*</artifactId>
|
||||
</exclusion>
|
||||
</exclusions>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.hbase</groupId>
|
||||
<artifactId>hbase-client</artifactId>
|
||||
<version>${hbase.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.hbase</groupId>
|
||||
<artifactId>hbase-protocol</artifactId>
|
||||
<version>${hbase.version}</version>
|
||||
</dependency>
|
||||
|
||||
<!-- zookeeper -->
|
||||
<dependency>
|
||||
<groupId>org.apache.curator</groupId>
|
||||
|
||||
@@ -155,6 +155,8 @@
|
||||
<exclude>META-INF/*.DSA</exclude>
|
||||
<exclude>META-INF/*.RSA</exclude>
|
||||
<exclude>META-INF/services/javax.*</exclude>
|
||||
<exclude>**/*.proto</exclude>
|
||||
<exclude>hbase-webapps/**</exclude>
|
||||
</excludes>
|
||||
</filter>
|
||||
</filters>
|
||||
@@ -198,17 +200,114 @@
|
||||
<include>com.fasterxml.jackson.core:jackson-annotations</include>
|
||||
<include>com.fasterxml.jackson.core:jackson-core</include>
|
||||
<include>com.fasterxml.jackson.core:jackson-databind</include>
|
||||
<include>org.apache.htrace:htrace-core</include>
|
||||
<include>org.apache.hbase:hbase-common</include>
|
||||
<include>org.apache.hbase:hbase-client</include>
|
||||
<include>org.apache.hbase:hbase-protocol</include>
|
||||
<include>org.apache.hbase:hbase-hadoop-compat</include>
|
||||
<include>org.apache.hbase:hbase-hadoop2-compat</include>
|
||||
<include>org.apache.hbase:hbase-metrics</include>
|
||||
<include>org.apache.hbase:hbase-metrics-api</include>
|
||||
<include>org.apache.hbase:hbase-protocol-shaded</include>
|
||||
<include>org.apache.hbase:hbase-server</include>
|
||||
<include>org.apache.hbase.thirdparty:hbase-shaded-miscellaneous</include>
|
||||
<include>org.apache.hbase.thirdparty:hbase-shaded-netty</include>
|
||||
<include>org.apache.hbase.thirdparty:hbase-shaded-protobuf</include>
|
||||
<include>org.apache.htrace:htrace-core4</include>
|
||||
<include>com.esotericsoftware:kryo-shaded</include>
|
||||
<include>com.esotericsoftware:minlog</include>
|
||||
<include>commons-io:commons-io</include>
|
||||
<include>log4j:log4j</include>
|
||||
<include>org.objenesis:objenesis</include>
|
||||
</includes>
|
||||
</artifactSet>
|
||||
<relocations>
|
||||
<relocation>
|
||||
<pattern>org.apache.commons.io.</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.commons.io.</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.hbase.</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.hbase.</shadedPattern>
|
||||
<excludes>
|
||||
<exclude>org.apache.hadoop.hbase.KeyValue$KeyComparator</exclude>
|
||||
</excludes>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hbase.</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hbase.</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.htrace.</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.htrace.</shadedPattern>
|
||||
</relocation>
|
||||
<!-- The classes below in org.apache.hadoop.metrics2 package come from
|
||||
hbase-hadoop-compat and hbase-hadoop2-compat, which have to be shaded one by one,
|
||||
instead of shading all classes under org.apache.hadoop.metrics2 including ones
|
||||
from hadoop. -->
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.MetricHistogram</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.MetricHistogram
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.MetricsExecutor</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.MetricsExecutor
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.impl.JmxCacheBuster</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.impl.JmxCacheBuster</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.lib.DefaultMetricsSystemHelper</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.lib.DefaultMetricsSystemHelper
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.lib.DynamicMetricsRegistry</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.lib.DynamicMetricsRegistry
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.lib.MetricsExecutorImpl</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.lib.MetricsExecutorImpl
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.lib.MutableFastCounter</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableFastCounter
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.lib.MutableHistogram</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableHistogram
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.lib.MutableRangeHistogram</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableRangeHistogram
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.lib.MutableSizeHistogram</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableSizeHistogram
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.lib.MutableTimeHistogram</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableTimeHistogram
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.util.MetricQuantile</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.util.MetricQuantile
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.util.MetricSampleQuantiles</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.util.MetricSampleQuantiles
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
</relocations>
|
||||
</configuration>
|
||||
<executions>
|
||||
<execution>
|
||||
|
||||
@@ -62,6 +62,7 @@
|
||||
<resource>META-INF/LICENSE</resource>
|
||||
<file>target/classes/META-INF/LICENSE</file>
|
||||
</transformer>
|
||||
<transformer implementation="org.apache.maven.plugins.shade.resource.ServicesResourceTransformer"/>
|
||||
</transformers>
|
||||
<artifactSet>
|
||||
<includes>
|
||||
@@ -76,22 +77,52 @@
|
||||
<include>com.esotericsoftware:minlog</include>
|
||||
<include>org.apache.hbase:hbase-common</include>
|
||||
<include>org.apache.hbase:hbase-client</include>
|
||||
<include>org.apache.hbase:hbase-protocol</include>
|
||||
<include>org.apache.hbase:hbase-hadoop-compat</include>
|
||||
<include>org.apache.hbase:hbase-hadoop2-compat</include>
|
||||
<include>org.apache.hbase:hbase-metrics</include>
|
||||
<include>org.apache.hbase:hbase-metrics-api</include>
|
||||
<include>org.apache.hbase:hbase-protocol-shaded</include>
|
||||
<include>org.apache.hbase:hbase-server</include>
|
||||
<include>org.apache.hbase:hbase-annotations</include>
|
||||
<include>org.apache.htrace:htrace-core</include>
|
||||
<include>org.apache.hbase.thirdparty:hbase-shaded-protobuf</include>
|
||||
<include>org.apache.hbase.thirdparty:hbase-shaded-netty</include>
|
||||
<include>org.apache.hbase.thirdparty:hbase-shaded-miscellaneous</include>
|
||||
<include>org.apache.htrace:htrace-core4</include>
|
||||
<include>com.yammer.metrics:metrics-core</include>
|
||||
<include>com.google.guava:guava</include>
|
||||
<include>commons-lang:commons-lang</include>
|
||||
<include>commons-io:commons-io</include>
|
||||
<include>com.google.protobuf:protobuf-java</include>
|
||||
</includes>
|
||||
</artifactSet>
|
||||
<relocations>
|
||||
|
||||
<relocation>
|
||||
<pattern>org.apache.parquet.avro.</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.parquet.avro.</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.avro.</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.avro.</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.commons.io.</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.commons.io.</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.hbase.</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.hbase.</shadedPattern>
|
||||
<excludes>
|
||||
<exclude>org.apache.hadoop.hbase.KeyValue$KeyComparator</exclude>
|
||||
</excludes>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hbase.</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hbase.</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.htrace.</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.htrace.</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.codehaus.jackson.</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.codehaus.jackson.</shadedPattern>
|
||||
@@ -124,6 +155,74 @@
|
||||
<pattern>com.google.protobuf.</pattern>
|
||||
<shadedPattern>${trino.bundle.bootstrap.shade.prefix}com.google.protobuf.</shadedPattern>
|
||||
</relocation>
|
||||
<!-- The classes below in org.apache.hadoop.metrics2 package come from
|
||||
hbase-hadoop-compat and hbase-hadoop2-compat, which have to be shaded one by one,
|
||||
instead of shading all classes under org.apache.hadoop.metrics2 including ones
|
||||
from hadoop. -->
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.MetricHistogram</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.MetricHistogram
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.MetricsExecutor</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.MetricsExecutor
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.impl.JmxCacheBuster</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.impl.JmxCacheBuster</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.lib.DefaultMetricsSystemHelper</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.lib.DefaultMetricsSystemHelper
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.lib.DynamicMetricsRegistry</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.lib.DynamicMetricsRegistry
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.lib.MetricsExecutorImpl</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.lib.MetricsExecutorImpl
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.lib.MutableFastCounter</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableFastCounter
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.lib.MutableHistogram</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableHistogram
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.lib.MutableRangeHistogram</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableRangeHistogram
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.lib.MutableSizeHistogram</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableSizeHistogram
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.lib.MutableTimeHistogram</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableTimeHistogram
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.util.MetricQuantile</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.util.MetricQuantile
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.util.MetricSampleQuantiles</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.util.MetricSampleQuantiles
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
</relocations>
|
||||
<createDependencyReducedPom>false</createDependencyReducedPom>
|
||||
<filters>
|
||||
@@ -134,6 +233,8 @@
|
||||
<exclude>META-INF/*.DSA</exclude>
|
||||
<exclude>META-INF/*.RSA</exclude>
|
||||
<exclude>META-INF/services/javax.*</exclude>
|
||||
<exclude>**/*.proto</exclude>
|
||||
<exclude>hbase-webapps/**</exclude>
|
||||
</excludes>
|
||||
</filter>
|
||||
</filters>
|
||||
@@ -155,71 +256,10 @@
|
||||
|
||||
<dependencies>
|
||||
<!-- Hoodie -->
|
||||
<dependency>
|
||||
<groupId>org.apache.hudi</groupId>
|
||||
<artifactId>hudi-common</artifactId>
|
||||
<version>${project.version}</version>
|
||||
<exclusions>
|
||||
<exclusion>
|
||||
<groupId>org.apache.hbase</groupId>
|
||||
<artifactId>hbase-server</artifactId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<groupId>org.apache.hbase</groupId>
|
||||
<artifactId>hbase-client</artifactId>
|
||||
</exclusion>
|
||||
</exclusions>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.hudi</groupId>
|
||||
<artifactId>hudi-hadoop-mr-bundle</artifactId>
|
||||
<version>${project.version}</version>
|
||||
<exclusions>
|
||||
<exclusion>
|
||||
<groupId>org.apache.hbase</groupId>
|
||||
<artifactId>hbase-server</artifactId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<groupId>org.apache.hbase</groupId>
|
||||
<artifactId>hbase-client</artifactId>
|
||||
</exclusion>
|
||||
</exclusions>
|
||||
</dependency>
|
||||
|
||||
<!-- HBase -->
|
||||
<dependency>
|
||||
<groupId>org.apache.hbase</groupId>
|
||||
<artifactId>hbase-common</artifactId>
|
||||
<version>${hbase.version}</version>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>org.apache.hbase</groupId>
|
||||
<artifactId>hbase-server</artifactId>
|
||||
<version>${hbase.version}</version>
|
||||
<scope>compile</scope>
|
||||
<exclusions>
|
||||
<exclusion>
|
||||
<groupId>org.apache.hbase</groupId>
|
||||
<artifactId>hbase-common</artifactId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<groupId>javax.servlet</groupId>
|
||||
<artifactId>*</artifactId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<groupId>org.codehaus.jackson</groupId>
|
||||
<artifactId>*</artifactId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<groupId>org.mortbay.jetty</groupId>
|
||||
<artifactId>*</artifactId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<groupId>tomcat</groupId>
|
||||
<artifactId>*</artifactId>
|
||||
</exclusion>
|
||||
</exclusions>
|
||||
</dependency>
|
||||
|
||||
<!-- Parquet -->
|
||||
|
||||
@@ -86,6 +86,7 @@
|
||||
<transformer implementation="org.apache.maven.plugins.shade.resource.AppendingTransformer">
|
||||
<resource>META-INF/services/org.apache.spark.sql.sources.DataSourceRegister</resource>
|
||||
</transformer>
|
||||
<transformer implementation="org.apache.maven.plugins.shade.resource.ServicesResourceTransformer"/>
|
||||
</transformers>
|
||||
<artifactSet>
|
||||
<includes>
|
||||
@@ -149,13 +150,21 @@
|
||||
|
||||
<include>org.apache.hbase:hbase-client</include>
|
||||
<include>org.apache.hbase:hbase-common</include>
|
||||
<include>org.apache.hbase:hbase-protocol</include>
|
||||
<include>org.apache.hbase:hbase-hadoop-compat</include>
|
||||
<include>org.apache.hbase:hbase-hadoop2-compat</include>
|
||||
<include>org.apache.hbase:hbase-metrics</include>
|
||||
<include>org.apache.hbase:hbase-metrics-api</include>
|
||||
<include>org.apache.hbase:hbase-protocol-shaded</include>
|
||||
<include>org.apache.hbase:hbase-server</include>
|
||||
<include>org.apache.htrace:htrace-core</include>
|
||||
<include>org.apache.hbase.thirdparty:hbase-shaded-miscellaneous</include>
|
||||
<include>org.apache.hbase.thirdparty:hbase-shaded-netty</include>
|
||||
<include>org.apache.hbase.thirdparty:hbase-shaded-protobuf</include>
|
||||
<include>org.apache.htrace:htrace-core4</include>
|
||||
<include>org.apache.curator:curator-framework</include>
|
||||
<include>org.apache.curator:curator-client</include>
|
||||
<include>org.apache.curator:curator-recipes</include>
|
||||
<include>commons-codec:commons-codec</include>
|
||||
<include>commons-io:commons-io</include>
|
||||
</includes>
|
||||
</artifactSet>
|
||||
<relocations>
|
||||
@@ -171,6 +180,25 @@
|
||||
<pattern>org.apache.hive.jdbc.</pattern>
|
||||
<shadedPattern>${utilities.bundle.hive.shade.prefix}org.apache.hive.jdbc.</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.commons.io.</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.commons.io.</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.hbase.</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.hbase.</shadedPattern>
|
||||
<excludes>
|
||||
<exclude>org.apache.hadoop.hbase.KeyValue$KeyComparator</exclude>
|
||||
</excludes>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hbase.</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hbase.</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.htrace.</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.htrace.</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.hive.metastore.</pattern>
|
||||
<shadedPattern>${utilities.bundle.hive.shade.prefix}org.apache.hadoop.hive.metastore.</shadedPattern>
|
||||
@@ -207,6 +235,74 @@
|
||||
<pattern>org.eclipse.jetty.</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.eclipse.jetty.</shadedPattern>
|
||||
</relocation>
|
||||
<!-- The classes below in org.apache.hadoop.metrics2 package come from
|
||||
hbase-hadoop-compat and hbase-hadoop2-compat, which have to be shaded one by one,
|
||||
instead of shading all classes under org.apache.hadoop.metrics2 including ones
|
||||
from hadoop. -->
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.MetricHistogram</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.MetricHistogram
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.MetricsExecutor</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.MetricsExecutor
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.impl.JmxCacheBuster</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.impl.JmxCacheBuster</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.lib.DefaultMetricsSystemHelper</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.lib.DefaultMetricsSystemHelper
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.lib.DynamicMetricsRegistry</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.lib.DynamicMetricsRegistry
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.lib.MetricsExecutorImpl</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.lib.MetricsExecutorImpl
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.lib.MutableFastCounter</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableFastCounter
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.lib.MutableHistogram</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableHistogram
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.lib.MutableRangeHistogram</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableRangeHistogram
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.lib.MutableSizeHistogram</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableSizeHistogram
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.lib.MutableTimeHistogram</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableTimeHistogram
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.util.MetricQuantile</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.util.MetricQuantile
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
<relocation>
|
||||
<pattern>org.apache.hadoop.metrics2.util.MetricSampleQuantiles</pattern>
|
||||
<shadedPattern>org.apache.hudi.org.apache.hadoop.metrics2.util.MetricSampleQuantiles
|
||||
</shadedPattern>
|
||||
</relocation>
|
||||
</relocations>
|
||||
<filters>
|
||||
<filter>
|
||||
@@ -216,6 +312,8 @@
|
||||
<exclude>META-INF/*.DSA</exclude>
|
||||
<exclude>META-INF/*.RSA</exclude>
|
||||
<exclude>META-INF/services/javax.*</exclude>
|
||||
<exclude>**/*.proto</exclude>
|
||||
<exclude>hbase-webapps/**</exclude>
|
||||
</excludes>
|
||||
</filter>
|
||||
</filters>
|
||||
@@ -339,51 +437,6 @@
|
||||
<scope>compile</scope>
|
||||
</dependency>
|
||||
|
||||
<!-- Hbase -->
|
||||
<dependency>
|
||||
<groupId>org.apache.hbase</groupId>
|
||||
<artifactId>hbase-common</artifactId>
|
||||
<version>${hbase.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.hbase</groupId>
|
||||
<artifactId>hbase-server</artifactId>
|
||||
<version>${hbase.version}</version>
|
||||
<scope>compile</scope>
|
||||
<exclusions>
|
||||
<exclusion>
|
||||
<groupId>org.apache.hbase</groupId>
|
||||
<artifactId>hbase-common</artifactId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<groupId>javax.servlet</groupId>
|
||||
<artifactId>*</artifactId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<groupId>org.codehaus.jackson</groupId>
|
||||
<artifactId>*</artifactId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<groupId>org.mortbay.jetty</groupId>
|
||||
<artifactId>*</artifactId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<groupId>tomcat</groupId>
|
||||
<artifactId>*</artifactId>
|
||||
</exclusion>
|
||||
</exclusions>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.hbase</groupId>
|
||||
<artifactId>hbase-client</artifactId>
|
||||
<version>${hbase.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.hbase</groupId>
|
||||
<artifactId>hbase-protocol</artifactId>
|
||||
<version>${hbase.version}</version>
|
||||
</dependency>
|
||||
|
||||
<!-- zookeeper -->
|
||||
<dependency>
|
||||
<groupId>org.apache.curator</groupId>
|
||||
|
||||
8
pom.xml
8
pom.xml
@@ -103,7 +103,7 @@
|
||||
<log4j.test.version>2.17.0</log4j.test.version>
|
||||
<slf4j.version>1.7.30</slf4j.version>
|
||||
<joda.version>2.9.9</joda.version>
|
||||
<hadoop.version>2.7.3</hadoop.version>
|
||||
<hadoop.version>2.10.1</hadoop.version>
|
||||
<hive.groupid>org.apache.hive</hive.groupid>
|
||||
<hive.version>2.3.1</hive.version>
|
||||
<hive.exec.classifier>core</hive.exec.classifier>
|
||||
@@ -138,7 +138,8 @@
|
||||
<thrift.version>0.12.0</thrift.version>
|
||||
<jetty.version>9.4.15.v20190215</jetty.version>
|
||||
<htrace.version>3.1.0-incubating</htrace.version>
|
||||
<hbase.version>1.2.3</hbase.version>
|
||||
<hbase.version>2.4.9</hbase.version>
|
||||
<hbase-thirdparty.version>3.5.1</hbase-thirdparty.version>
|
||||
<codehaus-jackson.version>1.9.13</codehaus-jackson.version>
|
||||
<h2.version>1.4.199</h2.version>
|
||||
<awaitility.version>3.1.2</awaitility.version>
|
||||
@@ -170,6 +171,7 @@
|
||||
<proto.version>3.17.3</proto.version>
|
||||
<protoc.version>3.11.4</protoc.version>
|
||||
<dynamodb.lockclient.version>1.1.0</dynamodb.lockclient.version>
|
||||
<zookeeper.version>3.5.7</zookeeper.version>
|
||||
<dynamodb-local.port>8000</dynamodb-local.port>
|
||||
<dynamodb-local.endpoint>http://localhost:${dynamodb-local.port}</dynamodb-local.endpoint>
|
||||
</properties>
|
||||
@@ -1529,7 +1531,7 @@
|
||||
<link>https://docs.spring.io/spring-shell/docs/1.2.0.RELEASE</link>
|
||||
<link>https://fasterxml.github.io/jackson-databind/javadoc/2.6</link>
|
||||
<link>https://hadoop.apache.org/docs/r${hadoop.version}/api</link>
|
||||
<link>https://hbase.apache.org/1.2/apidocs</link>
|
||||
<link>https://hbase.apache.org/2.4/apidocs</link>
|
||||
<link>https://hive.apache.org/javadocs/r2.3.6/api</link>
|
||||
<link>https://javadoc.io/static/io.javalin/javalin/2.3.0</link>
|
||||
<link>https://javadoc.io/doc/org.apache.parquet/parquet-avro/${parquet.version}</link>
|
||||
|
||||
Reference in New Issue
Block a user