From 5bbecaed7fdf6e2f95ac5df94291098f557f8469 Mon Sep 17 00:00:00 2001 From: Sagar Sumit Date: Wed, 17 Nov 2021 20:57:20 +0530 Subject: [PATCH 01/36] [HUDI-2783] Upgrade HBase Fix some unit tests Resolve dependency issue Upgrade Hadoop to 2.10.1 and fix HFile inline reader test Separate hbase shaded version for presto bundle Resolve hbase dep conflicts in flink, utilities and hadoop-mr bundles --- .../io/storage/HoodieFileWriterFactory.java | 4 +- .../hudi/io/storage/HoodieHFileConfig.java | 14 ++-- .../hudi/io/storage/HoodieHFileWriter.java | 30 ++++---- hudi-client/hudi-spark-client/pom.xml | 6 ++ .../hudi/testutils/HoodieClientTestUtils.java | 8 ++- hudi-common/pom.xml | 1 - .../bootstrap/index/HFileBootstrapIndex.java | 17 +++-- .../log/AbstractHoodieLogRecordReader.java | 3 + .../table/log/block/HoodieHFileDataBlock.java | 1 + .../io/storage/HoodieHBaseKVComparator.java | 4 +- .../hudi/io/storage/HoodieHFileReader.java | 60 ++++++++-------- .../TestInLineFileSystemHFileInLining.java | 68 +++++++++---------- packaging/hudi-flink-bundle/pom.xml | 28 +++++++- packaging/hudi-hadoop-mr-bundle/pom.xml | 46 ++++++++++++- packaging/hudi-spark-bundle/pom.xml | 32 ++++++++- packaging/hudi-utilities-bundle/pom.xml | 38 ++++++++++- pom.xml | 9 ++- 17 files changed, 259 insertions(+), 110 deletions(-) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieFileWriterFactory.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieFileWriterFactory.java index 38db1cde41226..f5e6514c5f34f 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieFileWriterFactory.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieFileWriterFactory.java @@ -85,12 +85,12 @@ private static HoodieFi TaskContextSupplier taskContextSupplier) throws IOException { BloomFilter filter = createBloomFilter(config); - HoodieHFileConfig hfileConfig = new HoodieHFileConfig(hoodieTable.getHadoopConf(), + HoodieHFileConfig hFileConfig = new HoodieHFileConfig(hoodieTable.getHadoopConf(), config.getHFileCompressionAlgorithm(), config.getHFileBlockSize(), config.getHFileMaxFileSize(), HoodieHFileReader.KEY_FIELD_NAME, PREFETCH_ON_OPEN, CACHE_DATA_IN_L1, DROP_BEHIND_CACHE_COMPACTION, filter, HFILE_COMPARATOR); - return new HoodieHFileWriter<>(instantTime, path, hfileConfig, schema, taskContextSupplier, config.populateMetaFields()); + return new HoodieHFileWriter<>(instantTime, path, hFileConfig, schema, taskContextSupplier, config.populateMetaFields()); } private static HoodieFileWriter newOrcFileWriter( diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileConfig.java index 1079566b782f1..255921cf7dd67 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileConfig.java @@ -21,14 +21,14 @@ 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; @@ -56,7 +56,7 @@ public HoodieHFileConfig(Configuration hadoopConf, Compression.Algorithm compres this.cacheDataInL1 = cacheDataInL1; this.dropBehindCacheCompaction = dropBehindCacheCompaction; this.bloomFilter = bloomFilter; - this.hfileComparator = hfileComparator; + this.hFileComparator = hFileComparator; this.keyFieldName = keyFieldName; } @@ -96,8 +96,8 @@ public BloomFilter getBloomFilter() { return bloomFilter; } - public KeyValue.KVComparator getHfileComparator() { - return hfileComparator; + public CellComparator getHFileComparator() { + return hFileComparator; } public String getKeyFieldName() { diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileWriter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileWriter.java index 2ad6d7f9220b0..409e84e06153d 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileWriter.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileWriter.java @@ -59,7 +59,7 @@ public class HoodieHFileWriter + + org.apache.zookeeper + zookeeper + ${zookeeper.version} + test + diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestUtils.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestUtils.java index 05d7f99446e94..8a3abfd6e1cbf 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestUtils.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestUtils.java @@ -66,6 +66,8 @@ 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 +243,9 @@ public static Stream readHFile(JavaSparkContext jsc, String[] pat Schema schema = null; for (String path : paths) { try { - HFile.Reader reader = HFile.createReader(fs, new Path(path), cacheConfig, fs.getConf()); + HFile.Reader reader = HFile.createReader(fs, new Path(path), cacheConfig, true, 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 +254,7 @@ public static Stream readHFile(JavaSparkContext jsc, String[] pat } 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()); diff --git a/hudi-common/pom.xml b/hudi-common/pom.xml index 1a558aeae3326..b00b82f86c172 100644 --- a/hudi-common/pom.xml +++ b/hudi-common/pom.xml @@ -221,7 +221,6 @@ org.apache.hbase hbase-client ${hbase.version} - test diff --git a/hudi-common/src/main/java/org/apache/hudi/common/bootstrap/index/HFileBootstrapIndex.java b/hudi-common/src/main/java/org/apache/hudi/common/bootstrap/index/HFileBootstrapIndex.java index 3700d01a60ea6..7f36a47a4d24c 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/bootstrap/index/HFileBootstrapIndex.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/bootstrap/index/HFileBootstrapIndex.java @@ -37,6 +37,7 @@ 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 +179,7 @@ private static String getUserKeyFromCellKey(String cellKey) { 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 HFile.createReader(fileSystem, new HFilePathForReader(hFilePath), new CacheConfig(conf), true, conf); } catch (IOException ioe) { throw new HoodieIOException(ioe.getMessage(), ioe); } @@ -259,7 +258,7 @@ private void initIndexInfo() { private HoodieBootstrapIndexInfo fetchBootstrapIndexInfo() throws IOException { return TimelineMetadataUtils.deserializeAvroMetadata( - partitionIndexReader().loadFileInfo().get(INDEX_INFO_KEY), + partitionIndexReader().getHFileInfo().get(INDEX_INFO_KEY), HoodieBootstrapIndexInfo.class); } @@ -306,7 +305,7 @@ private List getAllKeys(HFileScanner scanner, Function convert 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 +527,13 @@ public void close() { @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 +580,6 @@ public String getName() { * 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 { } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/AbstractHoodieLogRecordReader.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/AbstractHoodieLogRecordReader.java index fa5117e41fa76..6a0b10fe07ea0 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/AbstractHoodieLogRecordReader.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/AbstractHoodieLogRecordReader.java @@ -424,6 +424,9 @@ private void processQueuedBlocksForInstant(Deque logBlocks, int 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: diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieHFileDataBlock.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieHFileDataBlock.java index 557a0db7cbfad..f302e8afa21dd 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieHFileDataBlock.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieHFileDataBlock.java @@ -95,6 +95,7 @@ protected byte[] serializeRecords(List records) throws IOExceptio HFileContext context = new HFileContextBuilder() .withBlockSize(DEFAULT_BLOCK_SIZE) .withCompression(compressionAlgorithm.get()) + .withCellComparator(new HoodieHBaseKVComparator()) .build(); Configuration conf = new Configuration(); diff --git a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHBaseKVComparator.java b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHBaseKVComparator.java index 2d4d96959e150..aaf1dcd7037b7 100644 --- a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHBaseKVComparator.java +++ b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHBaseKVComparator.java @@ -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 { } diff --git a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileReader.java b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileReader.java index 371da7675e992..6e4fed37e1ca4 100644 --- a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileReader.java +++ b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileReader.java @@ -19,14 +19,12 @@ 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; @@ -44,7 +42,11 @@ 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.io.hfile.ReaderContext; +import org.apache.hadoop.hbase.io.hfile.ReaderContextBuilder; +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; @@ -59,6 +61,8 @@ import org.apache.log4j.LogManager; import org.apache.log4j.Logger; +import static org.apache.hudi.common.fs.FSUtils.getFs; + public class HoodieHFileReader implements HoodieFileReader { private static final Logger LOG = LogManager.getLogger(HoodieHFileReader.class); private Path path; @@ -80,14 +84,14 @@ public class HoodieHFileReader implements HoodieFileRea 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 = HFile.createReader(getFs(path.toString(), configuration), path, cacheConfig, true, 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 = HFile.createReader(fs, path, cacheConfig, true configuration); } public HoodieHFileReader(byte[] content) throws IOException { @@ -95,30 +99,32 @@ public HoodieHFileReader(byte[] content) throws IOException { 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); + FSDataInputStreamWrapper stream = new FSDataInputStreamWrapper(fsdis); + ReaderContext context = new ReaderContextBuilder() + .withFilePath(path) + .withInputStreamWrapper(stream) + .withFileSize(getFs("hoodie", conf).getFileStatus(path).getLen()) + .withFileSystem(stream.getHfs()) + .withPrimaryReplicaReader(true) + .withReaderType(ReaderContext.ReaderType.STREAM) + .build(); + HFileInfo fileInfo = new HFileInfo(context, conf); + this.reader = HFile.createReader(context, fileInfo, new CacheConfig(conf), conf); + fileInfo.initMetaAndIndex(reader); } @Override public String[] readMinMaxRecordKeys() { - try { - Map 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 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 +139,10 @@ public void withSchema(Schema schema) { @Override public BloomFilter readBloomFilter() { - Map 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).getBufferReadOnly(); byte[] filterBytes = new byte[serializedFilter.remaining()]; serializedFilter.get(filterBytes); // read the bytes that were written return BloomFilterFactory.fromString(new String(filterBytes), @@ -206,7 +212,7 @@ private List> readAllRecords(Schema writerSchema, Schema readerS final HFileScanner scanner = reader.getScanner(false, false); if (scanner.seekTo()) { do { - Cell c = scanner.getKeyValue(); + Cell c = scanner.getCell(); final Pair keyAndRecordPair = getRecordFromCell(c, writerSchema, readerSchema, keyFieldSchema); recordList.add(keyAndRecordPair); } while (scanner.next()); @@ -250,7 +256,7 @@ public List> readRecords(List keys) throws IOException { */ public List> readRecords(List keys, Schema schema) throws IOException { this.schema = schema; - reader.loadFileInfo(); + reader.getHFileInfo(); List> records = new ArrayList<>(); for (String key: keys) { Option value = getRecordByKey(key, schema); @@ -310,7 +316,7 @@ public boolean hasNext() { // 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 keyAndRecordPair = getRecordFromCell(scanner.getKeyValue(), getSchema(), readerSchema, keyFieldSchema); + final Pair keyAndRecordPair = getRecordFromCell(scanner.getCell(), getSchema(), readerSchema, keyFieldSchema); this.next = keyAndRecordPair.getSecond(); } } @@ -331,7 +337,7 @@ public R next() { } R retVal = this.next; if (scanner.next()) { - final Pair keyAndRecordPair = getRecordFromCell(scanner.getKeyValue(), getSchema(), readerSchema, keyFieldSchema); + final Pair keyAndRecordPair = getRecordFromCell(scanner.getCell(), getSchema(), readerSchema, keyFieldSchema); this.next = keyAndRecordPair.getSecond(); } else { this.next = null; @@ -371,7 +377,7 @@ public Option getRecordByKey(String key, Schema readerSchema) throws IOException } 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()); } diff --git a/hudi-common/src/test/java/org/apache/hudi/common/fs/inline/TestInLineFileSystemHFileInLining.java b/hudi-common/src/test/java/org/apache/hudi/common/fs/inline/TestInLineFileSystemHFileInLining.java index cc59b46024792..f09ecf76b2d88 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/fs/inline/TestInLineFileSystemHFileInLining.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/fs/inline/TestInLineFileSystemHFileInLining.java @@ -19,12 +19,12 @@ package org.apache.hudi.common.fs.inline; import org.apache.hudi.common.testutils.FileSystemTestUtils; -import org.apache.hudi.io.storage.HoodieHBaseKVComparator; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataInputStream; 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 +39,12 @@ 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 +58,12 @@ */ public class TestInLineFileSystemHFileInLining { + private static final String LOCAL_FORMATTER = "%010d"; + private static final String VALUE_PREFIX = "value"; + private static final int MIN_BLOCK_SIZE = 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 +91,11 @@ public void testSimpleInlineFileSystem() throws IOException { CacheConfig cacheConf = new CacheConfig(inMemoryConf); FSDataOutputStream fout = createFSOutput(outerInMemFSPath, inMemoryConf); HFileContext meta = new HFileContextBuilder() - .withBlockSize(minBlockSize) + .withBlockSize(MIN_BLOCK_SIZE).withCellComparator(COMPARATOR) .build(); HFile.Writer writer = HFile.getWriterFactory(inMemoryConf, cacheConf) .withOutputStream(fout) .withFileContext(meta) - .withComparator(new HoodieHBaseKVComparator()) .create(); writeRecords(writer); @@ -110,9 +112,9 @@ public void testSimpleInlineFileSystem() throws IOException { InLineFileSystem inlineFileSystem = (InLineFileSystem) inlinePath.getFileSystem(inlineConf); FSDataInputStream fin = inlineFileSystem.open(inlinePath); - HFile.Reader reader = HFile.createReader(inlineFileSystem, inlinePath, cacheConf, inlineConf); + HFile.Reader reader = HFile.createReader(inlineFileSystem, inlinePath, cacheConf, true, inlineConf); // Load up the index. - reader.loadFileInfo(); + reader.getHFileInfo(); // 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 void testSimpleInlineFileSystem() throws IOException { Set 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()); + assertArrayEquals(Arrays.copyOfRange(keyValue.getRowArray(), keyValue.getRowOffset(), keyValue.getRowOffset() + keyValue.getRowLength()), 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 @@ private Set getRandomValidRowIds(int count) { } 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 @@ private void writeRecords(HFile.Writer writer) throws IOException { 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,27 @@ private void readAllRecords(HFileScanner scanner) throws IOException { } // 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(); + assertArrayEquals(Arrays.copyOfRange(kv.getRowArray(), kv.getRowOffset(), kv.getRowOffset() + kv.getRowLength()), 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 { diff --git a/packaging/hudi-flink-bundle/pom.xml b/packaging/hudi-flink-bundle/pom.xml index 30ee37a4ecf2f..35c45da14d9b3 100644 --- a/packaging/hudi-flink-bundle/pom.xml +++ b/packaging/hudi-flink-bundle/pom.xml @@ -149,7 +149,11 @@ org.apache.hbase:hbase-common org.apache.hbase:hbase-client org.apache.hbase:hbase-server - org.apache.hbase:hbase-protocol + org.apache.hbase:hbase-protocol-shaded + org.apache.hbase.thirdparty:hbase-shaded-miscellaneous + org.apache.hbase.thirdparty:hbase-shaded-netty + org.apache.hbase.thirdparty:hbase-shaded-protobuf + org.apache.hbase:hbase-hadoop-compat org.apache.htrace:htrace-core commons-codec:commons-codec @@ -603,9 +607,29 @@ org.apache.hbase - hbase-protocol + hbase-protocol-shaded ${hbase.version} + + org.apache.hbase.thirdparty + hbase-shaded-miscellaneous + ${hbase-thirdparty.version} + + + org.apache.hbase.thirdparty + hbase-shaded-netty + ${hbase-thirdparty.version} + + + org.apache.hbase + hbase-hadoop-compat + ${hbase.version} + + + org.apache.hbase.thirdparty + hbase-shaded-protobuf + ${hbase-thirdparty.version} + org.apache.htrace htrace-core diff --git a/packaging/hudi-hadoop-mr-bundle/pom.xml b/packaging/hudi-hadoop-mr-bundle/pom.xml index f6215b1e017a5..d8cf3165bf50b 100644 --- a/packaging/hudi-hadoop-mr-bundle/pom.xml +++ b/packaging/hudi-hadoop-mr-bundle/pom.xml @@ -74,8 +74,12 @@ com.esotericsoftware:minlog org.apache.hbase:hbase-common org.apache.hbase:hbase-client - org.apache.hbase:hbase-protocol + org.apache.hbase:hbase-protocol-shaded org.apache.hbase:hbase-server + org.apache.hbase.thirdparty:hbase-shaded-miscellaneous + org.apache.hbase.thirdparty:hbase-shaded-netty + org.apache.hbase.thirdparty:hbase-shaded-protobuf + org.apache.hbase:hbase-hadoop-compat org.apache.htrace:htrace-core com.yammer.metrics:metrics-core com.google.guava:guava @@ -180,6 +184,12 @@ org.apache.hbase hbase-common ${hbase.version} + + + guava + com.google.guava + + @@ -188,6 +198,10 @@ ${hbase.version} compile + + guava + com.google.guava + org.apache.hbase hbase-common @@ -210,5 +224,35 @@ + + org.apache.hbase + hbase-client + ${hbase.version} + + + org.apache.hbase + hbase-protocol-shaded + ${hbase.version} + + + org.apache.hbase.thirdparty + hbase-shaded-miscellaneous + ${hbase-thirdparty.version} + + + org.apache.hbase.thirdparty + hbase-shaded-netty + ${hbase-thirdparty.version} + + + org.apache.hbase + hbase-hadoop-compat + ${hbase.version} + + + org.apache.hbase.thirdparty + hbase-shaded-protobuf + ${hbase-thirdparty.version} + diff --git a/packaging/hudi-spark-bundle/pom.xml b/packaging/hudi-spark-bundle/pom.xml index a877d10a586a8..6fc6babecc483 100644 --- a/packaging/hudi-spark-bundle/pom.xml +++ b/packaging/hudi-spark-bundle/pom.xml @@ -116,8 +116,12 @@ org.apache.hbase:hbase-client org.apache.hbase:hbase-common - org.apache.hbase:hbase-protocol + org.apache.hbase:hbase-protocol-shaded org.apache.hbase:hbase-server + org.apache.hbase.thirdparty:hbase-shaded-miscellaneous + org.apache.hbase.thirdparty:hbase-shaded-netty + org.apache.hbase.thirdparty:hbase-shaded-protobuf + org.apache.hbase:hbase-hadoop-compat org.apache.htrace:htrace-core org.apache.curator:curator-framework org.apache.curator:curator-client @@ -333,6 +337,10 @@ ${hbase.version} compile + + guava + com.google.guava + org.apache.hbase hbase-common @@ -362,9 +370,29 @@ org.apache.hbase - hbase-protocol + hbase-protocol-shaded + ${hbase.version} + + + org.apache.hbase.thirdparty + hbase-shaded-miscellaneous + ${hbase-thirdparty.version} + + + org.apache.hbase.thirdparty + hbase-shaded-netty + ${hbase-thirdparty.version} + + + org.apache.hbase + hbase-hadoop-compat ${hbase.version} + + org.apache.hbase.thirdparty + hbase-shaded-protobuf + ${hbase-thirdparty.version} + diff --git a/packaging/hudi-utilities-bundle/pom.xml b/packaging/hudi-utilities-bundle/pom.xml index 0685baee0a9ff..5613dc35c54f0 100644 --- a/packaging/hudi-utilities-bundle/pom.xml +++ b/packaging/hudi-utilities-bundle/pom.xml @@ -149,8 +149,12 @@ org.apache.hbase:hbase-client org.apache.hbase:hbase-common - org.apache.hbase:hbase-protocol + org.apache.hbase:hbase-protocol-shaded org.apache.hbase:hbase-server + org.apache.hbase.thirdparty:hbase-shaded-miscellaneous + org.apache.hbase.thirdparty:hbase-shaded-netty + org.apache.hbase.thirdparty:hbase-shaded-protobuf + org.apache.hbase:hbase-hadoop-compat org.apache.htrace:htrace-core org.apache.curator:curator-framework org.apache.curator:curator-client @@ -344,6 +348,12 @@ org.apache.hbase hbase-common ${hbase.version} + + + guava + com.google.guava + + org.apache.hbase @@ -351,6 +361,10 @@ ${hbase.version} compile + + guava + com.google.guava + org.apache.hbase hbase-common @@ -380,9 +394,29 @@ org.apache.hbase - hbase-protocol + hbase-protocol-shaded + ${hbase.version} + + + org.apache.hbase.thirdparty + hbase-shaded-miscellaneous + ${hbase-thirdparty.version} + + + org.apache.hbase.thirdparty + hbase-shaded-netty + ${hbase-thirdparty.version} + + + org.apache.hbase + hbase-hadoop-compat ${hbase.version} + + org.apache.hbase.thirdparty + hbase-shaded-protobuf + ${hbase-thirdparty.version} + diff --git a/pom.xml b/pom.xml index c61d5ef8f3a75..9ca5cfe13b0df 100644 --- a/pom.xml +++ b/pom.xml @@ -103,7 +103,7 @@ 2.17.0 1.7.30 2.9.9 - 2.7.3 + 2.10.1 org.apache.hive 2.3.1 core @@ -138,7 +138,9 @@ 0.12.0 9.4.15.v20190215 3.1.0-incubating - 1.2.3 + 2.4.7 + 1.2.3 + 3.5.1 1.9.13 1.4.199 3.1.2 @@ -170,6 +172,7 @@ 3.17.3 3.11.4 1.1.0 + 3.5.7 8000 http://localhost:${dynamodb-local.port} @@ -1529,7 +1532,7 @@ https://docs.spring.io/spring-shell/docs/1.2.0.RELEASE https://fasterxml.github.io/jackson-databind/javadoc/2.6 https://hadoop.apache.org/docs/r${hadoop.version}/api - https://hbase.apache.org/1.2/apidocs + https://hbase.apache.org/2.4/apidocs https://hive.apache.org/javadocs/r2.3.6/api https://javadoc.io/static/io.javalin/javalin/2.3.0 https://javadoc.io/doc/org.apache.parquet/parquet-avro/${parquet.version} From 25896f3401d876acf54ebc200e6aa3ad461e4274 Mon Sep 17 00:00:00 2001 From: Sagar Sumit Date: Wed, 8 Dec 2021 20:52:35 +0530 Subject: [PATCH 02/36] Fix metablock buffered read in hfile reader Diasble access time validation --- .../main/java/org/apache/hudi/io/storage/HoodieHFileReader.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileReader.java b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileReader.java index 6e4fed37e1ca4..a59101e694a11 100644 --- a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileReader.java +++ b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileReader.java @@ -142,7 +142,7 @@ public BloomFilter readBloomFilter() { HFileInfo fileInfo; try { fileInfo = reader.getHFileInfo(); - ByteBuff serializedFilter = reader.getMetaBlock(KEY_BLOOM_FILTER_META_BLOCK, false).getBufferReadOnly(); + 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), From ca393d201913a66729d4861f5647055e2d7fed67 Mon Sep 17 00:00:00 2001 From: Sagar Sumit Date: Mon, 3 Jan 2022 22:17:17 +0530 Subject: [PATCH 03/36] Replace hbase-shaded-server and shade some more deps --- packaging/hudi-presto-bundle/pom.xml | 18 +++++++++++++++++ packaging/hudi-trino-bundle/pom.xml | 29 ++++++++++++++++------------ 2 files changed, 35 insertions(+), 12 deletions(-) diff --git a/packaging/hudi-presto-bundle/pom.xml b/packaging/hudi-presto-bundle/pom.xml index 90c1087dcb4d2..d7f0167d36897 100644 --- a/packaging/hudi-presto-bundle/pom.xml +++ b/packaging/hudi-presto-bundle/pom.xml @@ -78,13 +78,23 @@ org.apache.hbase:hbase-protocol org.apache.hbase:hbase-server org.apache.htrace:htrace-core + org.apache.hbase:hbase-annotations + org.apache.hbase.thirdparty:hbase-shaded-protobuf + org.apache.hbase.thirdparty:hbase-shaded-netty + org.apache.hbase.thirdparty:hbase-shaded-miscellaneous + org.apache.htrace:htrace-core4 com.yammer.metrics:metrics-core com.google.guava:guava + commons-io:commons-io commons-lang:commons-lang com.google.protobuf:protobuf-java + + org.apache.parquet.avro. + org.apache.hudi.org.apache.parquet.avro. + org.apache.avro. org.apache.hudi.org.apache.avro. @@ -105,6 +115,10 @@ com.esotericsoftware.minlog. org.apache.hudi.com.esotericsoftware.minlog. + + org.apache.htrace. + ${trino.bundle.bootstrap.shade.prefix}org.apache.htrace. + com.yammer.metrics. org.apache.hudi.com.yammer.metrics. @@ -113,6 +127,10 @@ com.google.common. ${presto.bundle.bootstrap.shade.prefix}com.google.common. + + org.apache.commons.io. + ${trino.bundle.bootstrap.shade.prefix}org.apache.commons.io. + org.apache.commons.lang. ${presto.bundle.bootstrap.shade.prefix}org.apache.commons.lang. diff --git a/packaging/hudi-trino-bundle/pom.xml b/packaging/hudi-trino-bundle/pom.xml index adf73f1bb0b83..dffa9779ca2f6 100644 --- a/packaging/hudi-trino-bundle/pom.xml +++ b/packaging/hudi-trino-bundle/pom.xml @@ -79,15 +79,22 @@ org.apache.hbase:hbase-protocol org.apache.hbase:hbase-server org.apache.hbase:hbase-annotations - org.apache.htrace:htrace-core + org.apache.hbase.thirdparty:hbase-shaded-protobuf + org.apache.hbase.thirdparty:hbase-shaded-netty + org.apache.hbase.thirdparty:hbase-shaded-miscellaneous + org.apache.htrace:htrace-core4 com.yammer.metrics:metrics-core com.google.guava:guava + commons-io:commons-io commons-lang:commons-lang com.google.protobuf:protobuf-java - + + org.apache.parquet.avro. + org.apache.hudi.org.apache.parquet.avro. + org.apache.avro. org.apache.hudi.org.apache.avro. @@ -108,6 +115,10 @@ com.esotericsoftware.minlog. org.apache.hudi.com.esotericsoftware.minlog. + + org.apache.htrace. + ${trino.bundle.bootstrap.shade.prefix}org.apache.htrace. + com.yammer.metrics. org.apache.hudi.com.yammer.metrics. @@ -116,6 +127,10 @@ com.google.common. ${trino.bundle.bootstrap.shade.prefix}com.google.common. + + org.apache.commons.io. + ${trino.bundle.bootstrap.shade.prefix}org.apache.commons.io. + org.apache.commons.lang. ${trino.bundle.bootstrap.shade.prefix}org.apache.commons.lang. @@ -159,16 +174,6 @@ org.apache.hudi hudi-common ${project.version} - - - org.apache.hbase - hbase-server - - - org.apache.hbase - hbase-client - - org.apache.hudi From 1295af062a34a9b1fbcd671a1a8843fc7d3761a0 Mon Sep 17 00:00:00 2001 From: Y Ethan Guo Date: Thu, 17 Feb 2022 14:39:33 -0800 Subject: [PATCH 04/36] Rebase master --- .../io/storage/HoodieFileWriterFactory.java | 4 +- .../hudi/io/storage/HoodieHFileConfig.java | 8 +-- .../hudi/io/storage/HoodieHFileWriter.java | 35 ++++++----- .../table/log/block/HoodieHFileDataBlock.java | 25 ++++---- .../hudi/io/storage/HoodieHFileReader.java | 62 +++++++++---------- pom.xml | 2 +- 6 files changed, 69 insertions(+), 67 deletions(-) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieFileWriterFactory.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieFileWriterFactory.java index f5e6514c5f34f..38db1cde41226 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieFileWriterFactory.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieFileWriterFactory.java @@ -85,12 +85,12 @@ private static HoodieFi TaskContextSupplier taskContextSupplier) throws IOException { BloomFilter filter = createBloomFilter(config); - HoodieHFileConfig hFileConfig = new HoodieHFileConfig(hoodieTable.getHadoopConf(), + HoodieHFileConfig hfileConfig = new HoodieHFileConfig(hoodieTable.getHadoopConf(), config.getHFileCompressionAlgorithm(), config.getHFileBlockSize(), config.getHFileMaxFileSize(), HoodieHFileReader.KEY_FIELD_NAME, PREFETCH_ON_OPEN, CACHE_DATA_IN_L1, DROP_BEHIND_CACHE_COMPACTION, filter, HFILE_COMPARATOR); - return new HoodieHFileWriter<>(instantTime, path, hFileConfig, schema, taskContextSupplier, config.populateMetaFields()); + return new HoodieHFileWriter<>(instantTime, path, hfileConfig, schema, taskContextSupplier, config.populateMetaFields()); } private static HoodieFileWriter newOrcFileWriter( diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileConfig.java index 255921cf7dd67..5ce377901a4ba 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileConfig.java @@ -42,12 +42,12 @@ public class HoodieHFileConfig { private final boolean dropBehindCacheCompaction; private final Configuration hadoopConf; private final BloomFilter bloomFilter; - private final CellComparator 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, CellComparator hFileComparator) { + boolean dropBehindCacheCompaction, BloomFilter bloomFilter, CellComparator hfileComparator) { this.hadoopConf = hadoopConf; this.compressionAlgorithm = compressionAlgorithm; this.blockSize = blockSize; @@ -56,7 +56,7 @@ public HoodieHFileConfig(Configuration hadoopConf, Compression.Algorithm compres this.cacheDataInL1 = cacheDataInL1; this.dropBehindCacheCompaction = dropBehindCacheCompaction; this.bloomFilter = bloomFilter; - this.hFileComparator = hFileComparator; + this.hfileComparator = hfileComparator; this.keyFieldName = keyFieldName; } @@ -97,7 +97,7 @@ public BloomFilter getBloomFilter() { } public CellComparator getHFileComparator() { - return hFileComparator; + return hfileComparator; } public String getKeyFieldName() { diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileWriter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileWriter.java index 409e84e06153d..0be0646c5d578 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileWriter.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileWriter.java @@ -25,6 +25,8 @@ 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.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; @@ -59,7 +59,7 @@ public class HoodieHFileWriter records) throws IOExceptio } 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) -> { diff --git a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileReader.java b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileReader.java index a59101e694a11..ef3937bc24f2b 100644 --- a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileReader.java +++ b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileReader.java @@ -18,16 +18,16 @@ package org.apache.hudi.io.storage; -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.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; @@ -48,20 +48,20 @@ import org.apache.hadoop.hbase.io.hfile.ReaderContextBuilder; 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 static org.apache.hudi.common.fs.FSUtils.getFs; +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 implements HoodieFileReader { private static final Logger LOG = LogManager.getLogger(HoodieHFileReader.class); @@ -84,14 +84,14 @@ public class HoodieHFileReader implements HoodieFileRea public HoodieHFileReader(Configuration configuration, Path path, CacheConfig cacheConfig) throws IOException { this.conf = configuration; this.path = path; - this.reader = HFile.createReader(getFs(path.toString(), configuration), path, cacheConfig, true, conf); + this.reader = HFile.createReader(FSUtils.getFs(path.toString(), configuration), path, cacheConfig, true, 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, true configuration); + this.reader = HFile.createReader(fs, path, cacheConfig, true, configuration); } public HoodieHFileReader(byte[] content) throws IOException { @@ -101,13 +101,13 @@ public HoodieHFileReader(byte[] content) throws IOException { FSDataInputStream fsdis = new FSDataInputStream(bis); FSDataInputStreamWrapper stream = new FSDataInputStreamWrapper(fsdis); ReaderContext context = new ReaderContextBuilder() - .withFilePath(path) - .withInputStreamWrapper(stream) - .withFileSize(getFs("hoodie", conf).getFileStatus(path).getLen()) - .withFileSystem(stream.getHfs()) - .withPrimaryReplicaReader(true) - .withReaderType(ReaderContext.ReaderType.STREAM) - .build(); + .withFilePath(path) + .withInputStreamWrapper(stream) + .withFileSize(FSUtils.getFs("hoodie", conf).getFileStatus(path).getLen()) + .withFileSystem(stream.getHfs()) + .withPrimaryReplicaReader(true) + .withReaderType(ReaderContext.ReaderType.STREAM) + .build(); HFileInfo fileInfo = new HFileInfo(context, conf); this.reader = HFile.createReader(context, fileInfo, new CacheConfig(conf), conf); fileInfo.initMetaAndIndex(reader); @@ -269,7 +269,7 @@ public List> readRecords(List keys, Schema schema) throw public ClosableIterator getRecordIterator(List keys, Schema schema) throws IOException { this.schema = schema; - reader.loadFileInfo(); + reader.getHFileInfo(); Iterator iterator = keys.iterator(); return new ClosableIterator() { private R next; diff --git a/pom.xml b/pom.xml index 9ca5cfe13b0df..6985755309253 100644 --- a/pom.xml +++ b/pom.xml @@ -138,7 +138,7 @@ 0.12.0 9.4.15.v20190215 3.1.0-incubating - 2.4.7 + 2.4.9 1.2.3 3.5.1 1.9.13 From d2f94af65c220e442771da15c1d5bd846efb79b7 Mon Sep 17 00:00:00 2001 From: Y Ethan Guo Date: Thu, 17 Feb 2022 16:11:56 -0800 Subject: [PATCH 05/36] Fix HoodieHFileReader --- .../java/org/apache/hudi/io/storage/HoodieHFileWriter.java | 1 - .../java/org/apache/hudi/io/storage/HoodieHFileReader.java | 7 +++++-- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileWriter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileWriter.java index 0be0646c5d578..5dcd2e0a32e51 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileWriter.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileWriter.java @@ -80,7 +80,6 @@ public HoodieHFileWriter(String instantTime, Path file, HoodieHFileConfig hfileC Configuration conf = FSUtils.registerFileSystem(file, hfileConfig.getHadoopConf()); this.file = HoodieWrapperFileSystem.convertToHoodiePath(file, conf); this.fs = (HoodieWrapperFileSystem) this.file.getFileSystem(conf); - this.hfileConfig = hfileConfig; this.schema = schema; this.keyFieldSchema = Option.ofNullable(schema.getField(hfileConfig.getKeyFieldName())); diff --git a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileReader.java b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileReader.java index ef3937bc24f2b..5c861c9cc7a26 100644 --- a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileReader.java +++ b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileReader.java @@ -39,6 +39,7 @@ import org.apache.hadoop.fs.Seekable; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.fs.HFileSystem; import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper; import org.apache.hadoop.hbase.io.hfile.CacheConfig; import org.apache.hadoop.hbase.io.hfile.HFile; @@ -100,11 +101,13 @@ public HoodieHFileReader(byte[] content) throws IOException { SeekableByteArrayInputStream bis = new SeekableByteArrayInputStream(content); FSDataInputStream fsdis = new FSDataInputStream(bis); FSDataInputStreamWrapper stream = new FSDataInputStreamWrapper(fsdis); + FileSystem fs = FSUtils.getFs("hoodie", conf); + HFileSystem hfs = (fs instanceof HFileSystem) ? (HFileSystem) fs : new HFileSystem(fs); ReaderContext context = new ReaderContextBuilder() .withFilePath(path) .withInputStreamWrapper(stream) - .withFileSize(FSUtils.getFs("hoodie", conf).getFileStatus(path).getLen()) - .withFileSystem(stream.getHfs()) + .withFileSize(content.length) + .withFileSystem(hfs) .withPrimaryReplicaReader(true) .withReaderType(ReaderContext.ReaderType.STREAM) .build(); From 9dfe1cf8ff3ffff62b63f729ef86004f61cde91d Mon Sep 17 00:00:00 2001 From: Y Ethan Guo Date: Thu, 17 Feb 2022 17:31:10 -0800 Subject: [PATCH 06/36] Revert hadoop upgrade --- pom.xml | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/pom.xml b/pom.xml index 6985755309253..43b3a08589e12 100644 --- a/pom.xml +++ b/pom.xml @@ -103,7 +103,7 @@ 2.17.0 1.7.30 2.9.9 - 2.10.1 + 2.7.3 org.apache.hive 2.3.1 core @@ -139,7 +139,6 @@ 9.4.15.v20190215 3.1.0-incubating 2.4.9 - 1.2.3 3.5.1 1.9.13 1.4.199 From b0b0e16c5d0c07886b2c4862708db3d28fe24d6e Mon Sep 17 00:00:00 2001 From: Y Ethan Guo Date: Fri, 18 Feb 2022 18:26:54 -0800 Subject: [PATCH 07/36] Fix hudi-hadoop-mr-bundle to include more dependencies and address CI failures --- ...ker-compose_hadoop284_hive233_spark244.yml | 4 +- docker/hoodie/hadoop/hive_base/Dockerfile | 3 + hudi-aws/pom.xml | 19 +++++ hudi-client/hudi-client-common/pom.xml | 39 ++++++++++ hudi-client/hudi-java-client/pom.xml | 20 +++++ hudi-common/pom.xml | 3 +- .../org/apache/hudi/integ/ITTestBase.java | 18 ++++- packaging/hudi-hadoop-mr-bundle/pom.xml | 75 ++----------------- packaging/hudi-spark-bundle/pom.xml | 6 ++ 9 files changed, 111 insertions(+), 76 deletions(-) diff --git a/docker/compose/docker-compose_hadoop284_hive233_spark244.yml b/docker/compose/docker-compose_hadoop284_hive233_spark244.yml index 086004f121e97..572788f2bdfb9 100644 --- a/docker/compose/docker-compose_hadoop284_hive233_spark244.yml +++ b/docker/compose/docker-compose_hadoop284_hive233_spark244.yml @@ -86,7 +86,7 @@ services: container_name: hive-metastore-postgresql hivemetastore: - image: apachehudi/hudi-hadoop_2.8.4-hive_2.3.3:latest + image: yihua/hudi-hadoop_2.8.4-hive_2.3.3:latest hostname: hivemetastore container_name: hivemetastore links: @@ -109,7 +109,7 @@ services: - "namenode" hiveserver: - image: apachehudi/hudi-hadoop_2.8.4-hive_2.3.3:latest + image: yihua/hudi-hadoop_2.8.4-hive_2.3.3:latest hostname: hiveserver container_name: hiveserver env_file: diff --git a/docker/hoodie/hadoop/hive_base/Dockerfile b/docker/hoodie/hadoop/hive_base/Dockerfile index 7d04d94fc60cc..8c4fbb5886bf0 100644 --- a/docker/hoodie/hadoop/hive_base/Dockerfile +++ b/docker/hoodie/hadoop/hive_base/Dockerfile @@ -36,6 +36,9 @@ RUN echo "Hive URL is :${HIVE_URL}" && wget ${HIVE_URL} -O hive.tar.gz && \ wget https://jdbc.postgresql.org/download/postgresql-9.4.1212.jar -O $HIVE_HOME/lib/postgresql-jdbc.jar && \ rm hive.tar.gz && mkdir -p /var/hoodie/ws/docker/hoodie/hadoop/hive_base/target/ +RUN rm hive/lib/hbase* +RUN rm hive/lib/commons-io-2.4.jar + #Spark should be compiled with Hive to be able to use it #hive-site.xml should be copied to $SPARK_HOME/conf folder diff --git a/hudi-aws/pom.xml b/hudi-aws/pom.xml index d44a389a61f66..4abbd119a0d55 100644 --- a/hudi-aws/pom.xml +++ b/hudi-aws/pom.xml @@ -48,6 +48,25 @@ + + org.apache.hadoop + hadoop-common + provided + + + org.mortbay.jetty + * + + + javax.servlet.jsp + * + + + javax.servlet + * + + + org.apache.hadoop hadoop-common diff --git a/hudi-client/hudi-client-common/pom.xml b/hudi-client/hudi-client-common/pom.xml index a55a136652728..9cdc1c40ad66a 100644 --- a/hudi-client/hudi-client-common/pom.xml +++ b/hudi-client/hudi-client-common/pom.xml @@ -117,6 +117,45 @@ + + org.apache.hadoop + hadoop-common + provided + + + org.mortbay.jetty + * + + + javax.servlet.jsp + * + + + javax.servlet + * + + + + + org.apache.hadoop + hadoop-hdfs + test + + + + org.mortbay.jetty + * + + + javax.servlet.jsp + * + + + javax.servlet + * + + + org.apache.hadoop hadoop-hdfs diff --git a/hudi-client/hudi-java-client/pom.xml b/hudi-client/hudi-java-client/pom.xml index 3471bfb8ba366..b299150c6e3e0 100644 --- a/hudi-client/hudi-java-client/pom.xml +++ b/hudi-client/hudi-java-client/pom.xml @@ -122,6 +122,26 @@ test + + org.apache.hadoop + hadoop-hdfs + test + + + + org.mortbay.jetty + * + + + javax.servlet.jsp + * + + + javax.servlet + * + + + org.apache.hadoop hadoop-hdfs diff --git a/hudi-common/pom.xml b/hudi-common/pom.xml index b00b82f86c172..0cd72c33464fd 100644 --- a/hudi-common/pom.xml +++ b/hudi-common/pom.xml @@ -221,13 +221,14 @@ org.apache.hbase hbase-client ${hbase.version} + test org.apache.hbase hbase-server ${hbase.version} - + compile diff --git a/hudi-integ-test/src/test/java/org/apache/hudi/integ/ITTestBase.java b/hudi-integ-test/src/test/java/org/apache/hudi/integ/ITTestBase.java index 4c0265ce90f64..f2a5395af32ea 100644 --- a/hudi-integ-test/src/test/java/org/apache/hudi/integ/ITTestBase.java +++ b/hudi-integ-test/src/test/java/org/apache/hudi/integ/ITTestBase.java @@ -221,9 +221,11 @@ private TestExecStartResultCallback executeCommandInDocker(String containerName, // Each execution of command(s) in docker should not be more than 15 mins. Otherwise, it is deemed stuck. We will // try to capture stdout and stderr of the stuck process. + LOG.error("containerName: " + containerName); + LOG.error("Command: " + Arrays.asList(command)); 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(); @@ -236,8 +238,11 @@ private TestExecStartResultCallback executeCommandInDocker(String containerName, int exitCode = dockerClient.inspectExecCmd(createCmdResponse.getId()).exec().getExitCode(); LOG.info("Exit code for command : " + exitCode); if (exitCode != 0) { - LOG.error("\n\n ###### Stdout #######\n" + callback.getStdout().toString()); + //LOG.error("\n\n ###### Stdout #######\n" + callback.getStdout().toString()); } + callback.getStderr().flush(); + callback.getStdout().flush(); + LOG.error("\n\n ###### Stdout #######\n" + callback.getStdout().toString()); LOG.error("\n\n ###### Stderr #######\n" + callback.getStderr().toString()); if (checkIfSucceed) { @@ -338,8 +343,8 @@ private void saveUpLogs() { executeCommandStringInDocker(HIVESERVER, "cat /tmp/root/hive.log | grep -i exception -A 10 -B 5", false).getStdout().toString(); String filePath = System.getProperty("java.io.tmpdir") + "/" + System.currentTimeMillis() + "-hive.log"; FileIOUtils.writeStringToFile(hiveLogStr, filePath); - LOG.info("Hive log saved up at : " + filePath); - LOG.info("<=========== Full hive log ===============>\n" + LOG.error("Hive log saved up at : " + filePath); + LOG.error("<=========== Full hive log ===============>\n" + "\n" + hiveLogStr + "\n <==========================================>"); } catch (Exception e) { @@ -356,6 +361,11 @@ void assertStdOutContains(Pair stdOutErr, String expectedOutput, String stdOutSingleSpaced = singleSpace(stdOutErr.getLeft()).replaceAll(" ", ""); expectedOutput = singleSpace(expectedOutput).replaceAll(" ", ""); + LOG.error("stdOutErr : " + stdOutErr.getLeft()); + LOG.error("stdOutErr.getRight : " + stdOutErr.getRight()); + LOG.error("stdOutSingleSpaced : " + stdOutSingleSpaced); + LOG.error("expectedOutput : " + expectedOutput); + int lastIndex = 0; int count = 0; while (lastIndex != -1) { diff --git a/packaging/hudi-hadoop-mr-bundle/pom.xml b/packaging/hudi-hadoop-mr-bundle/pom.xml index d8cf3165bf50b..4899b6e5c2095 100644 --- a/packaging/hudi-hadoop-mr-bundle/pom.xml +++ b/packaging/hudi-hadoop-mr-bundle/pom.xml @@ -74,12 +74,16 @@ com.esotericsoftware:minlog org.apache.hbase:hbase-common org.apache.hbase:hbase-client + org.apache.hbase:hbase-metrics + org.apache.hbase:hbase-metrics-api org.apache.hbase:hbase-protocol-shaded org.apache.hbase:hbase-server org.apache.hbase.thirdparty:hbase-shaded-miscellaneous org.apache.hbase.thirdparty:hbase-shaded-netty org.apache.hbase.thirdparty:hbase-shaded-protobuf org.apache.hbase:hbase-hadoop-compat + org.apache.hbase:hbase-hadoop2-compat + commons-io:commons-io org.apache.htrace:htrace-core com.yammer.metrics:metrics-core com.google.guava:guava @@ -172,81 +176,14 @@ compile - - org.apache.htrace - htrace-core - ${htrace.version} - compile - - - - - org.apache.hbase - hbase-common - ${hbase.version} - - - guava - com.google.guava - - - - - - org.apache.hbase - hbase-server - ${hbase.version} - compile - - - guava - com.google.guava - - - org.apache.hbase - hbase-common - - - javax.servlet - * - - - org.codehaus.jackson - * - - - org.mortbay.jetty - * - - - tomcat - * - - - org.apache.hbase - hbase-client - ${hbase.version} - - - org.apache.hbase - hbase-protocol-shaded + hbase-hadoop-compat ${hbase.version} - - org.apache.hbase.thirdparty - hbase-shaded-miscellaneous - ${hbase-thirdparty.version} - - - org.apache.hbase.thirdparty - hbase-shaded-netty - ${hbase-thirdparty.version} - org.apache.hbase - hbase-hadoop-compat + hbase-hadoop2-compat ${hbase.version} diff --git a/packaging/hudi-spark-bundle/pom.xml b/packaging/hudi-spark-bundle/pom.xml index 6fc6babecc483..12cd1abd077e9 100644 --- a/packaging/hudi-spark-bundle/pom.xml +++ b/packaging/hudi-spark-bundle/pom.xml @@ -122,6 +122,7 @@ org.apache.hbase.thirdparty:hbase-shaded-netty org.apache.hbase.thirdparty:hbase-shaded-protobuf org.apache.hbase:hbase-hadoop-compat + org.apache.hbase:hbase-hadoop2-compat org.apache.htrace:htrace-core org.apache.curator:curator-framework org.apache.curator:curator-client @@ -388,6 +389,11 @@ hbase-hadoop-compat ${hbase.version} + + org.apache.hbase + hbase-hadoop2-compat + ${hbase.version} + org.apache.hbase.thirdparty hbase-shaded-protobuf From 615325f4baf494696818e72811a8d2cc59bea9d6 Mon Sep 17 00:00:00 2001 From: Y Ethan Guo Date: Fri, 25 Feb 2022 13:47:44 -0800 Subject: [PATCH 08/36] Fix hudi-utilities-bundle --- packaging/hudi-utilities-bundle/pom.xml | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/packaging/hudi-utilities-bundle/pom.xml b/packaging/hudi-utilities-bundle/pom.xml index 5613dc35c54f0..1a63193abec42 100644 --- a/packaging/hudi-utilities-bundle/pom.xml +++ b/packaging/hudi-utilities-bundle/pom.xml @@ -149,12 +149,16 @@ org.apache.hbase:hbase-client org.apache.hbase:hbase-common + org.apache.hbase:hbase-metrics + org.apache.hbase:hbase-metrics-api org.apache.hbase:hbase-protocol-shaded org.apache.hbase:hbase-server org.apache.hbase.thirdparty:hbase-shaded-miscellaneous org.apache.hbase.thirdparty:hbase-shaded-netty org.apache.hbase.thirdparty:hbase-shaded-protobuf org.apache.hbase:hbase-hadoop-compat + org.apache.hbase:hbase-hadoop2-compat + commons-io:commons-io org.apache.htrace:htrace-core org.apache.curator:curator-framework org.apache.curator:curator-client From ff7635d92ae459947bc7840ee7647296c606db40 Mon Sep 17 00:00:00 2001 From: Y Ethan Guo Date: Fri, 25 Feb 2022 15:42:53 -0800 Subject: [PATCH 09/36] Fix hudi-common --- hudi-common/pom.xml | 19 ++++++++++++++++++- 1 file changed, 18 insertions(+), 1 deletion(-) diff --git a/hudi-common/pom.xml b/hudi-common/pom.xml index 0cd72c33464fd..c18eac7af4372 100644 --- a/hudi-common/pom.xml +++ b/hudi-common/pom.xml @@ -221,7 +221,24 @@ org.apache.hbase hbase-client ${hbase.version} - test + + + javax.servlet + * + + + org.codehaus.jackson + * + + + org.mortbay.jetty + * + + + tomcat + * + + From 079d58b5f75d5b7cdfe2aa9a612046d3a9e29cc5 Mon Sep 17 00:00:00 2001 From: Y Ethan Guo Date: Sat, 26 Feb 2022 10:41:35 -0800 Subject: [PATCH 10/36] Fix other bundles --- packaging/hudi-flink-bundle/pom.xml | 8 ++++++-- packaging/hudi-hadoop-mr-bundle/pom.xml | 2 +- packaging/hudi-presto-bundle/pom.xml | 8 +++++++- packaging/hudi-spark-bundle/pom.xml | 5 ++++- packaging/hudi-trino-bundle/pom.xml | 8 ++++++-- packaging/hudi-utilities-bundle/pom.xml | 2 +- 6 files changed, 25 insertions(+), 8 deletions(-) diff --git a/packaging/hudi-flink-bundle/pom.xml b/packaging/hudi-flink-bundle/pom.xml index 35c45da14d9b3..226d24e94f5a1 100644 --- a/packaging/hudi-flink-bundle/pom.xml +++ b/packaging/hudi-flink-bundle/pom.xml @@ -138,7 +138,7 @@ org.apache.hive:hive-service org.apache.hive:hive-service-rpc org.apache.hive:hive-exec - org.apache.hive:hive-standalone-metastore + org.apache.hive:hive-standalone-metastore org.apache.hive:hive-metastore org.apache.hive:hive-jdbc org.datanucleus:datanucleus-core @@ -148,13 +148,17 @@ org.apache.hbase:hbase-common org.apache.hbase:hbase-client + org.apache.hbase:hbase-metrics + org.apache.hbase:hbase-metrics-api org.apache.hbase:hbase-server org.apache.hbase:hbase-protocol-shaded org.apache.hbase.thirdparty:hbase-shaded-miscellaneous org.apache.hbase.thirdparty:hbase-shaded-netty org.apache.hbase.thirdparty:hbase-shaded-protobuf org.apache.hbase:hbase-hadoop-compat - org.apache.htrace:htrace-core + org.apache.hbase:hbase-hadoop2-compat + commons-io:commons-io + org.apache.htrace:htrace-core4 commons-codec:commons-codec diff --git a/packaging/hudi-hadoop-mr-bundle/pom.xml b/packaging/hudi-hadoop-mr-bundle/pom.xml index 4899b6e5c2095..c84c46ceadc1e 100644 --- a/packaging/hudi-hadoop-mr-bundle/pom.xml +++ b/packaging/hudi-hadoop-mr-bundle/pom.xml @@ -84,7 +84,7 @@ org.apache.hbase:hbase-hadoop-compat org.apache.hbase:hbase-hadoop2-compat commons-io:commons-io - org.apache.htrace:htrace-core + org.apache.htrace:htrace-core4 com.yammer.metrics:metrics-core com.google.guava:guava diff --git a/packaging/hudi-presto-bundle/pom.xml b/packaging/hudi-presto-bundle/pom.xml index d7f0167d36897..62579ef82ba07 100644 --- a/packaging/hudi-presto-bundle/pom.xml +++ b/packaging/hudi-presto-bundle/pom.xml @@ -75,13 +75,19 @@ com.esotericsoftware:minlog org.apache.hbase:hbase-common org.apache.hbase:hbase-client + org.apache.hbase:hbase-metrics + org.apache.hbase:hbase-metrics-api + org.apache.hbase:hbase-protocol - org.apache.hbase:hbase-server + org.apache.hbase:hbase-protocol-shaded org.apache.htrace:htrace-core org.apache.hbase:hbase-annotations org.apache.hbase.thirdparty:hbase-shaded-protobuf org.apache.hbase.thirdparty:hbase-shaded-netty org.apache.hbase.thirdparty:hbase-shaded-miscellaneous + org.apache.hbase:hbase-hadoop-compat + org.apache.hbase:hbase-hadoop2-compat + commons-io:commons-io org.apache.htrace:htrace-core4 com.yammer.metrics:metrics-core com.google.guava:guava diff --git a/packaging/hudi-spark-bundle/pom.xml b/packaging/hudi-spark-bundle/pom.xml index 12cd1abd077e9..6149b7bbd0265 100644 --- a/packaging/hudi-spark-bundle/pom.xml +++ b/packaging/hudi-spark-bundle/pom.xml @@ -116,6 +116,8 @@ org.apache.hbase:hbase-client org.apache.hbase:hbase-common + org.apache.hbase:hbase-metrics + org.apache.hbase:hbase-metrics-api org.apache.hbase:hbase-protocol-shaded org.apache.hbase:hbase-server org.apache.hbase.thirdparty:hbase-shaded-miscellaneous @@ -123,7 +125,8 @@ org.apache.hbase.thirdparty:hbase-shaded-protobuf org.apache.hbase:hbase-hadoop-compat org.apache.hbase:hbase-hadoop2-compat - org.apache.htrace:htrace-core + commons-io:commons-io + org.apache.htrace:htrace-core4 org.apache.curator:curator-framework org.apache.curator:curator-client org.apache.curator:curator-recipes diff --git a/packaging/hudi-trino-bundle/pom.xml b/packaging/hudi-trino-bundle/pom.xml index dffa9779ca2f6..8de546a7f896e 100644 --- a/packaging/hudi-trino-bundle/pom.xml +++ b/packaging/hudi-trino-bundle/pom.xml @@ -76,16 +76,20 @@ com.esotericsoftware:minlog org.apache.hbase:hbase-common org.apache.hbase:hbase-client - org.apache.hbase:hbase-protocol + org.apache.hbase:hbase-metrics + org.apache.hbase:hbase-metrics-api + org.apache.hbase:hbase-protocol-shaded org.apache.hbase:hbase-server org.apache.hbase:hbase-annotations org.apache.hbase.thirdparty:hbase-shaded-protobuf org.apache.hbase.thirdparty:hbase-shaded-netty org.apache.hbase.thirdparty:hbase-shaded-miscellaneous + org.apache.hbase:hbase-hadoop-compat + org.apache.hbase:hbase-hadoop2-compat + commons-io:commons-io org.apache.htrace:htrace-core4 com.yammer.metrics:metrics-core com.google.guava:guava - commons-io:commons-io commons-lang:commons-lang com.google.protobuf:protobuf-java diff --git a/packaging/hudi-utilities-bundle/pom.xml b/packaging/hudi-utilities-bundle/pom.xml index 1a63193abec42..13df9a32803a4 100644 --- a/packaging/hudi-utilities-bundle/pom.xml +++ b/packaging/hudi-utilities-bundle/pom.xml @@ -159,7 +159,7 @@ org.apache.hbase:hbase-hadoop-compat org.apache.hbase:hbase-hadoop2-compat commons-io:commons-io - org.apache.htrace:htrace-core + org.apache.htrace:htrace-core4 org.apache.curator:curator-framework org.apache.curator:curator-client org.apache.curator:curator-recipes From 6ae79af07ad188a04750eba6d3f5ddf168d50d22 Mon Sep 17 00:00:00 2001 From: Y Ethan Guo Date: Sun, 27 Feb 2022 22:48:39 -0800 Subject: [PATCH 11/36] Use hadoop 2.10.1 --- hudi-client/hudi-client-common/pom.xml | 39 -------------------------- hudi-client/hudi-spark-client/pom.xml | 6 ++++ hudi-common/pom.xml | 18 ------------ pom.xml | 2 +- 4 files changed, 7 insertions(+), 58 deletions(-) diff --git a/hudi-client/hudi-client-common/pom.xml b/hudi-client/hudi-client-common/pom.xml index 9cdc1c40ad66a..a55a136652728 100644 --- a/hudi-client/hudi-client-common/pom.xml +++ b/hudi-client/hudi-client-common/pom.xml @@ -117,45 +117,6 @@ - - org.apache.hadoop - hadoop-common - provided - - - org.mortbay.jetty - * - - - javax.servlet.jsp - * - - - javax.servlet - * - - - - - org.apache.hadoop - hadoop-hdfs - test - - - - org.mortbay.jetty - * - - - javax.servlet.jsp - * - - - javax.servlet - * - - - org.apache.hadoop hadoop-hdfs diff --git a/hudi-client/hudi-spark-client/pom.xml b/hudi-client/hudi-spark-client/pom.xml index 0688fedacc2ae..20cccda5ea420 100644 --- a/hudi-client/hudi-spark-client/pom.xml +++ b/hudi-client/hudi-spark-client/pom.xml @@ -90,6 +90,12 @@ + + org.apache.hadoop + hadoop-hdfs-client + ${hadoop.version} + test + org.apache.hbase hbase-testing-util diff --git a/hudi-common/pom.xml b/hudi-common/pom.xml index c18eac7af4372..b5b2a3a47a4ae 100644 --- a/hudi-common/pom.xml +++ b/hudi-common/pom.xml @@ -221,24 +221,6 @@ org.apache.hbase hbase-client ${hbase.version} - - - javax.servlet - * - - - org.codehaus.jackson - * - - - org.mortbay.jetty - * - - - tomcat - * - - diff --git a/pom.xml b/pom.xml index 43b3a08589e12..0c67085715bb2 100644 --- a/pom.xml +++ b/pom.xml @@ -103,7 +103,7 @@ 2.17.0 1.7.30 2.9.9 - 2.7.3 + 2.10.1 org.apache.hive 2.3.1 core From 8c681be52a17014b651d104764663ed0af6ddecf Mon Sep 17 00:00:00 2001 From: Y Ethan Guo Date: Mon, 28 Feb 2022 12:09:23 -0800 Subject: [PATCH 12/36] Shade dependencies in hudi-hadoop-mr-bundle --- packaging/hudi-hadoop-mr-bundle/pom.xml | 30 ++++++++++++++++++++++--- 1 file changed, 27 insertions(+), 3 deletions(-) diff --git a/packaging/hudi-hadoop-mr-bundle/pom.xml b/packaging/hudi-hadoop-mr-bundle/pom.xml index c84c46ceadc1e..c815592c90dc4 100644 --- a/packaging/hudi-hadoop-mr-bundle/pom.xml +++ b/packaging/hudi-hadoop-mr-bundle/pom.xml @@ -74,6 +74,8 @@ com.esotericsoftware:minlog org.apache.hbase:hbase-common org.apache.hbase:hbase-client + org.apache.hbase:hbase-hadoop-compat + org.apache.hbase:hbase-hadoop2-compat org.apache.hbase:hbase-metrics org.apache.hbase:hbase-metrics-api org.apache.hbase:hbase-protocol-shaded @@ -81,12 +83,10 @@ org.apache.hbase.thirdparty:hbase-shaded-miscellaneous org.apache.hbase.thirdparty:hbase-shaded-netty org.apache.hbase.thirdparty:hbase-shaded-protobuf - org.apache.hbase:hbase-hadoop-compat - org.apache.hbase:hbase-hadoop2-compat - commons-io:commons-io org.apache.htrace:htrace-core4 com.yammer.metrics:metrics-core com.google.guava:guava + commons-io:commons-io @@ -110,6 +110,22 @@ org.apache.avro. org.apache.hudi.org.apache.avro. + + org.apache.commons.io + org.apache.hudi.org.apache.commons.io + + + org.apache.hadoop.hbase. + org.apache.hudi.org.apache.hadoop.hbase. + + + org.apache.hbase. + org.apache.hudi.org.apache.hbase. + + + org.apache.htrace. + org.apache.hudi.org.apache.htrace. + org.apache.parquet.avro. org.apache.hudi.org.apache.parquet.avro. @@ -127,7 +143,15 @@ META-INF/*.SF META-INF/*.DSA META-INF/*.RSA + META-INF/maven/com.google.protobuf/** + META-INF/maven/commons-io/** + META-INF/maven/org.apache.hbase/** + META-INF/maven/org.apache.hbase.thirdparty/** + META-INF/maven/org.apache.htrace/** META-INF/services/javax.* + META-INF/services/org.apache.hadoop.hbase.* + **/*.proto + hbase-webapps/** From 324c1abe7c35ce11e9179dd47437da2dfb8ae248 Mon Sep 17 00:00:00 2001 From: Y Ethan Guo Date: Mon, 28 Feb 2022 12:33:38 -0800 Subject: [PATCH 13/36] Shade dependencies in other bundles --- packaging/hudi-flink-bundle/pom.xml | 30 ++++++++++++++++++--- packaging/hudi-hadoop-mr-bundle/pom.xml | 4 +-- packaging/hudi-presto-bundle/pom.xml | 35 ++++++++++++++++++++----- packaging/hudi-spark-bundle/pom.xml | 30 ++++++++++++++++++--- packaging/hudi-trino-bundle/pom.xml | 30 ++++++++++++++++++--- packaging/hudi-utilities-bundle/pom.xml | 30 ++++++++++++++++++--- 6 files changed, 138 insertions(+), 21 deletions(-) diff --git a/packaging/hudi-flink-bundle/pom.xml b/packaging/hudi-flink-bundle/pom.xml index 226d24e94f5a1..d7f67a0a0c98b 100644 --- a/packaging/hudi-flink-bundle/pom.xml +++ b/packaging/hudi-flink-bundle/pom.xml @@ -148,6 +148,8 @@ org.apache.hbase:hbase-common org.apache.hbase:hbase-client + org.apache.hbase:hbase-hadoop-compat + org.apache.hbase:hbase-hadoop2-compat org.apache.hbase:hbase-metrics org.apache.hbase:hbase-metrics-api org.apache.hbase:hbase-server @@ -155,11 +157,9 @@ org.apache.hbase.thirdparty:hbase-shaded-miscellaneous org.apache.hbase.thirdparty:hbase-shaded-netty org.apache.hbase.thirdparty:hbase-shaded-protobuf - org.apache.hbase:hbase-hadoop-compat - org.apache.hbase:hbase-hadoop2-compat - commons-io:commons-io org.apache.htrace:htrace-core4 commons-codec:commons-codec + commons-io:commons-io @@ -171,6 +171,22 @@ org.apache.avro. ${flink.bundle.shade.prefix}org.apache.avro. + + org.apache.commons.io. + org.apache.hudi.org.apache.commons.io. + + + org.apache.hadoop.hbase. + org.apache.hudi.org.apache.hadoop.hbase. + + + org.apache.hbase. + org.apache.hudi.org.apache.hbase. + + + org.apache.htrace. + org.apache.hudi.org.apache.htrace. + com.yammer.metrics. ${flink.bundle.shade.prefix}com.yammer.metrics. @@ -208,7 +224,15 @@ META-INF/*.SF META-INF/*.DSA META-INF/*.RSA + META-INF/maven/com.google.protobuf/** + META-INF/maven/commons-io/** + META-INF/maven/org.apache.hbase/** + META-INF/maven/org.apache.hbase.thirdparty/** + META-INF/maven/org.apache.htrace/** META-INF/services/javax.* + META-INF/services/org.apache.hadoop.hbase.* + **/*.proto + hbase-webapps/** diff --git a/packaging/hudi-hadoop-mr-bundle/pom.xml b/packaging/hudi-hadoop-mr-bundle/pom.xml index c815592c90dc4..4cf50241d87ba 100644 --- a/packaging/hudi-hadoop-mr-bundle/pom.xml +++ b/packaging/hudi-hadoop-mr-bundle/pom.xml @@ -111,8 +111,8 @@ org.apache.hudi.org.apache.avro. - org.apache.commons.io - org.apache.hudi.org.apache.commons.io + org.apache.commons.io. + org.apache.hudi.org.apache.commons.io. org.apache.hadoop.hbase. diff --git a/packaging/hudi-presto-bundle/pom.xml b/packaging/hudi-presto-bundle/pom.xml index 62579ef82ba07..04dfefe6a6d50 100644 --- a/packaging/hudi-presto-bundle/pom.xml +++ b/packaging/hudi-presto-bundle/pom.xml @@ -75,19 +75,16 @@ com.esotericsoftware:minlog org.apache.hbase:hbase-common org.apache.hbase:hbase-client + org.apache.hbase:hbase-hadoop-compat + org.apache.hbase:hbase-hadoop2-compat org.apache.hbase:hbase-metrics org.apache.hbase:hbase-metrics-api - org.apache.hbase:hbase-protocol org.apache.hbase:hbase-protocol-shaded - org.apache.htrace:htrace-core org.apache.hbase:hbase-annotations - org.apache.hbase.thirdparty:hbase-shaded-protobuf - org.apache.hbase.thirdparty:hbase-shaded-netty org.apache.hbase.thirdparty:hbase-shaded-miscellaneous - org.apache.hbase:hbase-hadoop-compat - org.apache.hbase:hbase-hadoop2-compat - commons-io:commons-io + org.apache.hbase.thirdparty:hbase-shaded-netty + org.apache.hbase.thirdparty:hbase-shaded-protobuf org.apache.htrace:htrace-core4 com.yammer.metrics:metrics-core com.google.guava:guava @@ -105,6 +102,22 @@ org.apache.avro. org.apache.hudi.org.apache.avro. + + org.apache.commons.io + org.apache.hudi.org.apache.commons.io + + + org.apache.hadoop.hbase. + org.apache.hudi.org.apache.hadoop.hbase. + + + org.apache.hbase. + org.apache.hudi.org.apache.hbase. + + + org.apache.htrace. + org.apache.hudi.org.apache.htrace. + org.codehaus.jackson. org.apache.hudi.org.codehaus.jackson. @@ -162,8 +175,16 @@ META-INF/*.SF META-INF/*.DSA META-INF/*.RSA + META-INF/maven/com.google.protobuf/** + META-INF/maven/commons-io/** + META-INF/maven/org.apache.hbase/** + META-INF/maven/org.apache.hbase.thirdparty/** + META-INF/maven/org.apache.htrace/** META-INF/services/javax.* + META-INF/services/org.apache.hadoop.hbase.* + **/*.proto com/esotericsoftware/reflectasm/** + hbase-webapps/** stringBehavior.avsc diff --git a/packaging/hudi-spark-bundle/pom.xml b/packaging/hudi-spark-bundle/pom.xml index 6149b7bbd0265..25aa3cc444338 100644 --- a/packaging/hudi-spark-bundle/pom.xml +++ b/packaging/hudi-spark-bundle/pom.xml @@ -116,6 +116,8 @@ org.apache.hbase:hbase-client org.apache.hbase:hbase-common + org.apache.hbase:hbase-hadoop-compat + org.apache.hbase:hbase-hadoop2-compat org.apache.hbase:hbase-metrics org.apache.hbase:hbase-metrics-api org.apache.hbase:hbase-protocol-shaded @@ -123,14 +125,12 @@ org.apache.hbase.thirdparty:hbase-shaded-miscellaneous org.apache.hbase.thirdparty:hbase-shaded-netty org.apache.hbase.thirdparty:hbase-shaded-protobuf - org.apache.hbase:hbase-hadoop-compat - org.apache.hbase:hbase-hadoop2-compat - commons-io:commons-io org.apache.htrace:htrace-core4 org.apache.curator:curator-framework org.apache.curator:curator-client org.apache.curator:curator-recipes commons-codec:commons-codec + commons-io:commons-io @@ -142,6 +142,22 @@ com.beust.jcommander. org.apache.hudi.com.beust.jcommander. + + org.apache.commons.io + org.apache.hudi.org.apache.commons.io + + + org.apache.hadoop.hbase. + org.apache.hudi.org.apache.hadoop.hbase. + + + org.apache.hbase. + org.apache.hudi.org.apache.hbase. + + + org.apache.htrace. + org.apache.hudi.org.apache.htrace. + org.apache.spark.sql.avro. ${spark.bundle.spark.shade.prefix}org.apache.spark.sql.avro. @@ -199,7 +215,15 @@ META-INF/*.SF META-INF/*.DSA META-INF/*.RSA + META-INF/maven/com.google.protobuf/** + META-INF/maven/commons-io/** + META-INF/maven/org.apache.hbase/** + META-INF/maven/org.apache.hbase.thirdparty/** + META-INF/maven/org.apache.htrace/** META-INF/services/javax.* + META-INF/services/org.apache.hadoop.hbase.* + **/*.proto + hbase-webapps/** diff --git a/packaging/hudi-trino-bundle/pom.xml b/packaging/hudi-trino-bundle/pom.xml index 8de546a7f896e..0b372950cf9d4 100644 --- a/packaging/hudi-trino-bundle/pom.xml +++ b/packaging/hudi-trino-bundle/pom.xml @@ -76,6 +76,8 @@ com.esotericsoftware:minlog org.apache.hbase:hbase-common org.apache.hbase:hbase-client + org.apache.hbase:hbase-hadoop-compat + org.apache.hbase:hbase-hadoop2-compat org.apache.hbase:hbase-metrics org.apache.hbase:hbase-metrics-api org.apache.hbase:hbase-protocol-shaded @@ -84,13 +86,11 @@ org.apache.hbase.thirdparty:hbase-shaded-protobuf org.apache.hbase.thirdparty:hbase-shaded-netty org.apache.hbase.thirdparty:hbase-shaded-miscellaneous - org.apache.hbase:hbase-hadoop-compat - org.apache.hbase:hbase-hadoop2-compat - commons-io:commons-io org.apache.htrace:htrace-core4 com.yammer.metrics:metrics-core com.google.guava:guava commons-lang:commons-lang + commons-io:commons-io com.google.protobuf:protobuf-java @@ -103,6 +103,22 @@ org.apache.avro. org.apache.hudi.org.apache.avro. + + org.apache.commons.io + org.apache.hudi.org.apache.commons.io + + + org.apache.hadoop.hbase. + org.apache.hudi.org.apache.hadoop.hbase. + + + org.apache.hbase. + org.apache.hudi.org.apache.hbase. + + + org.apache.htrace. + org.apache.hudi.org.apache.htrace. + org.codehaus.jackson. org.apache.hudi.org.codehaus.jackson. @@ -152,7 +168,15 @@ META-INF/*.SF META-INF/*.DSA META-INF/*.RSA + META-INF/maven/com.google.protobuf/** + META-INF/maven/commons-io/** + META-INF/maven/org.apache.hbase/** + META-INF/maven/org.apache.hbase.thirdparty/** + META-INF/maven/org.apache.htrace/** META-INF/services/javax.* + META-INF/services/org.apache.hadoop.hbase.* + **/*.proto + hbase-webapps/** diff --git a/packaging/hudi-utilities-bundle/pom.xml b/packaging/hudi-utilities-bundle/pom.xml index 13df9a32803a4..a20aa1399180c 100644 --- a/packaging/hudi-utilities-bundle/pom.xml +++ b/packaging/hudi-utilities-bundle/pom.xml @@ -149,6 +149,8 @@ org.apache.hbase:hbase-client org.apache.hbase:hbase-common + org.apache.hbase:hbase-hadoop-compat + org.apache.hbase:hbase-hadoop2-compat org.apache.hbase:hbase-metrics org.apache.hbase:hbase-metrics-api org.apache.hbase:hbase-protocol-shaded @@ -156,14 +158,12 @@ org.apache.hbase.thirdparty:hbase-shaded-miscellaneous org.apache.hbase.thirdparty:hbase-shaded-netty org.apache.hbase.thirdparty:hbase-shaded-protobuf - org.apache.hbase:hbase-hadoop-compat - org.apache.hbase:hbase-hadoop2-compat - commons-io:commons-io org.apache.htrace:htrace-core4 org.apache.curator:curator-framework org.apache.curator:curator-client org.apache.curator:curator-recipes commons-codec:commons-codec + commons-io:commons-io @@ -179,6 +179,22 @@ org.apache.hive.jdbc. ${utilities.bundle.hive.shade.prefix}org.apache.hive.jdbc. + + org.apache.commons.io + org.apache.hudi.org.apache.commons.io + + + org.apache.hadoop.hbase. + org.apache.hudi.org.apache.hadoop.hbase. + + + org.apache.hbase. + org.apache.hudi.org.apache.hbase. + + + org.apache.htrace. + org.apache.hudi.org.apache.htrace. + org.apache.hadoop.hive.metastore. ${utilities.bundle.hive.shade.prefix}org.apache.hadoop.hive.metastore. @@ -223,7 +239,15 @@ META-INF/*.SF META-INF/*.DSA META-INF/*.RSA + META-INF/maven/com.google.protobuf/** + META-INF/maven/commons-io/** + META-INF/maven/org.apache.hbase/** + META-INF/maven/org.apache.hbase.thirdparty/** + META-INF/maven/org.apache.htrace/** META-INF/services/javax.* + META-INF/services/org.apache.hadoop.hbase.* + **/*.proto + hbase-webapps/** From f9de8834c2b2ace2c1465e3b08e0c5a29746e19e Mon Sep 17 00:00:00 2001 From: Y Ethan Guo Date: Mon, 28 Feb 2022 12:56:59 -0800 Subject: [PATCH 14/36] Fix bundle shading --- ...docker-compose_hadoop284_hive233_spark244.yml | 4 ++-- docker/hoodie/hadoop/hive_base/Dockerfile | 3 --- packaging/hudi-presto-bundle/pom.xml | 16 ++-------------- packaging/hudi-spark-bundle/pom.xml | 4 ++-- packaging/hudi-trino-bundle/pom.xml | 12 ++---------- packaging/hudi-utilities-bundle/pom.xml | 4 ++-- 6 files changed, 10 insertions(+), 33 deletions(-) diff --git a/docker/compose/docker-compose_hadoop284_hive233_spark244.yml b/docker/compose/docker-compose_hadoop284_hive233_spark244.yml index 572788f2bdfb9..086004f121e97 100644 --- a/docker/compose/docker-compose_hadoop284_hive233_spark244.yml +++ b/docker/compose/docker-compose_hadoop284_hive233_spark244.yml @@ -86,7 +86,7 @@ services: container_name: hive-metastore-postgresql hivemetastore: - image: yihua/hudi-hadoop_2.8.4-hive_2.3.3:latest + image: apachehudi/hudi-hadoop_2.8.4-hive_2.3.3:latest hostname: hivemetastore container_name: hivemetastore links: @@ -109,7 +109,7 @@ services: - "namenode" hiveserver: - image: yihua/hudi-hadoop_2.8.4-hive_2.3.3:latest + image: apachehudi/hudi-hadoop_2.8.4-hive_2.3.3:latest hostname: hiveserver container_name: hiveserver env_file: diff --git a/docker/hoodie/hadoop/hive_base/Dockerfile b/docker/hoodie/hadoop/hive_base/Dockerfile index 8c4fbb5886bf0..7d04d94fc60cc 100644 --- a/docker/hoodie/hadoop/hive_base/Dockerfile +++ b/docker/hoodie/hadoop/hive_base/Dockerfile @@ -36,9 +36,6 @@ RUN echo "Hive URL is :${HIVE_URL}" && wget ${HIVE_URL} -O hive.tar.gz && \ wget https://jdbc.postgresql.org/download/postgresql-9.4.1212.jar -O $HIVE_HOME/lib/postgresql-jdbc.jar && \ rm hive.tar.gz && mkdir -p /var/hoodie/ws/docker/hoodie/hadoop/hive_base/target/ -RUN rm hive/lib/hbase* -RUN rm hive/lib/commons-io-2.4.jar - #Spark should be compiled with Hive to be able to use it #hive-site.xml should be copied to $SPARK_HOME/conf folder diff --git a/packaging/hudi-presto-bundle/pom.xml b/packaging/hudi-presto-bundle/pom.xml index 04dfefe6a6d50..48f483da6667e 100644 --- a/packaging/hudi-presto-bundle/pom.xml +++ b/packaging/hudi-presto-bundle/pom.xml @@ -103,8 +103,8 @@ org.apache.hudi.org.apache.avro. - org.apache.commons.io - org.apache.hudi.org.apache.commons.io + org.apache.commons.io. + org.apache.hudi.org.apache.commons.io. org.apache.hadoop.hbase. @@ -134,10 +134,6 @@ com.esotericsoftware.minlog. org.apache.hudi.com.esotericsoftware.minlog. - - org.apache.htrace. - ${trino.bundle.bootstrap.shade.prefix}org.apache.htrace. - com.yammer.metrics. org.apache.hudi.com.yammer.metrics. @@ -146,10 +142,6 @@ com.google.common. ${presto.bundle.bootstrap.shade.prefix}com.google.common. - - org.apache.commons.io. - ${trino.bundle.bootstrap.shade.prefix}org.apache.commons.io. - org.apache.commons.lang. ${presto.bundle.bootstrap.shade.prefix}org.apache.commons.lang. @@ -158,10 +150,6 @@ com.google.protobuf. ${presto.bundle.bootstrap.shade.prefix}com.google.protobuf. - - org.apache.htrace. - ${presto.bundle.bootstrap.shade.prefix}org.apache.htrace. - org.apache.parquet.avro. ${presto.bundle.bootstrap.shade.prefix}org.apache.parquet.avro. diff --git a/packaging/hudi-spark-bundle/pom.xml b/packaging/hudi-spark-bundle/pom.xml index 25aa3cc444338..37bc2db5639d0 100644 --- a/packaging/hudi-spark-bundle/pom.xml +++ b/packaging/hudi-spark-bundle/pom.xml @@ -143,8 +143,8 @@ org.apache.hudi.com.beust.jcommander. - org.apache.commons.io - org.apache.hudi.org.apache.commons.io + org.apache.commons.io. + org.apache.hudi.org.apache.commons.io. org.apache.hadoop.hbase. diff --git a/packaging/hudi-trino-bundle/pom.xml b/packaging/hudi-trino-bundle/pom.xml index 0b372950cf9d4..e6cb86c76942c 100644 --- a/packaging/hudi-trino-bundle/pom.xml +++ b/packaging/hudi-trino-bundle/pom.xml @@ -104,8 +104,8 @@ org.apache.hudi.org.apache.avro. - org.apache.commons.io - org.apache.hudi.org.apache.commons.io + org.apache.commons.io. + org.apache.hudi.org.apache.commons.io. org.apache.hadoop.hbase. @@ -135,10 +135,6 @@ com.esotericsoftware.minlog. org.apache.hudi.com.esotericsoftware.minlog. - - org.apache.htrace. - ${trino.bundle.bootstrap.shade.prefix}org.apache.htrace. - com.yammer.metrics. org.apache.hudi.com.yammer.metrics. @@ -147,10 +143,6 @@ com.google.common. ${trino.bundle.bootstrap.shade.prefix}com.google.common. - - org.apache.commons.io. - ${trino.bundle.bootstrap.shade.prefix}org.apache.commons.io. - org.apache.commons.lang. ${trino.bundle.bootstrap.shade.prefix}org.apache.commons.lang. diff --git a/packaging/hudi-utilities-bundle/pom.xml b/packaging/hudi-utilities-bundle/pom.xml index a20aa1399180c..3c2e34c296d68 100644 --- a/packaging/hudi-utilities-bundle/pom.xml +++ b/packaging/hudi-utilities-bundle/pom.xml @@ -180,8 +180,8 @@ ${utilities.bundle.hive.shade.prefix}org.apache.hive.jdbc. - org.apache.commons.io - org.apache.hudi.org.apache.commons.io + org.apache.commons.io. + org.apache.hudi.org.apache.commons.io. org.apache.hadoop.hbase. From 47519869d5fa1471be048f50b335041489773e42 Mon Sep 17 00:00:00 2001 From: Y Ethan Guo Date: Tue, 1 Mar 2022 17:49:34 -0800 Subject: [PATCH 15/36] Fix hbase shading --- packaging/hudi-flink-bundle/pom.xml | 66 +++++++++++++++++++++++- packaging/hudi-hadoop-mr-bundle/pom.xml | 68 ++++++++++++++++++++++++- packaging/hudi-presto-bundle/pom.xml | 66 +++++++++++++++++++++++- packaging/hudi-spark-bundle/pom.xml | 66 +++++++++++++++++++++++- packaging/hudi-trino-bundle/pom.xml | 66 +++++++++++++++++++++++- packaging/hudi-utilities-bundle/pom.xml | 66 +++++++++++++++++++++++- 6 files changed, 391 insertions(+), 7 deletions(-) diff --git a/packaging/hudi-flink-bundle/pom.xml b/packaging/hudi-flink-bundle/pom.xml index d7f67a0a0c98b..f79e3f43fc053 100644 --- a/packaging/hudi-flink-bundle/pom.xml +++ b/packaging/hudi-flink-bundle/pom.xml @@ -70,6 +70,7 @@ META-INF/LICENSE target/classes/META-INF/LICENSE + @@ -216,6 +217,70 @@ com.fasterxml.jackson. ${flink.bundle.shade.prefix}com.fasterxml.jackson. + + org.apache.hadoop.metrics2.MetricHistogram + org.apache.hudi.org.apache.hadoop.metrics2.MetricHistogram + + + + org.apache.hadoop.metrics2.MetricsExecutor + org.apache.hudi.org.apache.hadoop.metrics2.MetricsExecutor + + + + org.apache.hadoop.metrics2.impl.JmxCacheBuster + org.apache.hudi.org.apache.hadoop.metrics2.impl.JmxCacheBuster + + + org.apache.hadoop.metrics2.lib.DefaultMetricsSystemHelper + org.apache.hudi.org.apache.hadoop.metrics2.lib.DefaultMetricsSystemHelper + + + + org.apache.hadoop.metrics2.lib.DynamicMetricsRegistry + org.apache.hudi.org.apache.hadoop.metrics2.lib.DynamicMetricsRegistry + + + + org.apache.hadoop.metrics2.lib.MetricsExecutorImpl + org.apache.hudi.org.apache.hadoop.metrics2.lib.MetricsExecutorImpl + + + + org.apache.hadoop.metrics2.lib.MutableFastCounter + org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableFastCounter + + + + org.apache.hadoop.metrics2.lib.MutableHistogram + org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableHistogram + + + + org.apache.hadoop.metrics2.lib.MutableRangeHistogram + org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableRangeHistogram + + + + org.apache.hadoop.metrics2.lib.MutableSizeHistogram + org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableSizeHistogram + + + + org.apache.hadoop.metrics2.lib.MutableTimeHistogram + org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableTimeHistogram + + + + org.apache.hadoop.metrics2.util.MetricQuantile + org.apache.hudi.org.apache.hadoop.metrics2.util.MetricQuantile + + + + org.apache.hadoop.metrics2.util.MetricSampleQuantiles + org.apache.hudi.org.apache.hadoop.metrics2.util.MetricSampleQuantiles + + @@ -230,7 +295,6 @@ META-INF/maven/org.apache.hbase.thirdparty/** META-INF/maven/org.apache.htrace/** META-INF/services/javax.* - META-INF/services/org.apache.hadoop.hbase.* **/*.proto hbase-webapps/** diff --git a/packaging/hudi-hadoop-mr-bundle/pom.xml b/packaging/hudi-hadoop-mr-bundle/pom.xml index 4cf50241d87ba..6f8a95a47e66a 100644 --- a/packaging/hudi-hadoop-mr-bundle/pom.xml +++ b/packaging/hudi-hadoop-mr-bundle/pom.xml @@ -55,12 +55,13 @@ - true + true META-INF/LICENSE target/classes/META-INF/LICENSE + @@ -134,6 +135,70 @@ com.google.common. org.apache.hudi.com.google.common. + + org.apache.hadoop.metrics2.MetricHistogram + org.apache.hudi.org.apache.hadoop.metrics2.MetricHistogram + + + + org.apache.hadoop.metrics2.MetricsExecutor + org.apache.hudi.org.apache.hadoop.metrics2.MetricsExecutor + + + + org.apache.hadoop.metrics2.impl.JmxCacheBuster + org.apache.hudi.org.apache.hadoop.metrics2.impl.JmxCacheBuster + + + org.apache.hadoop.metrics2.lib.DefaultMetricsSystemHelper + org.apache.hudi.org.apache.hadoop.metrics2.lib.DefaultMetricsSystemHelper + + + + org.apache.hadoop.metrics2.lib.DynamicMetricsRegistry + org.apache.hudi.org.apache.hadoop.metrics2.lib.DynamicMetricsRegistry + + + + org.apache.hadoop.metrics2.lib.MetricsExecutorImpl + org.apache.hudi.org.apache.hadoop.metrics2.lib.MetricsExecutorImpl + + + + org.apache.hadoop.metrics2.lib.MutableFastCounter + org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableFastCounter + + + + org.apache.hadoop.metrics2.lib.MutableHistogram + org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableHistogram + + + + org.apache.hadoop.metrics2.lib.MutableRangeHistogram + org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableRangeHistogram + + + + org.apache.hadoop.metrics2.lib.MutableSizeHistogram + org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableSizeHistogram + + + + org.apache.hadoop.metrics2.lib.MutableTimeHistogram + org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableTimeHistogram + + + + org.apache.hadoop.metrics2.util.MetricQuantile + org.apache.hudi.org.apache.hadoop.metrics2.util.MetricQuantile + + + + org.apache.hadoop.metrics2.util.MetricSampleQuantiles + org.apache.hudi.org.apache.hadoop.metrics2.util.MetricSampleQuantiles + + false @@ -149,7 +214,6 @@ META-INF/maven/org.apache.hbase.thirdparty/** META-INF/maven/org.apache.htrace/** META-INF/services/javax.* - META-INF/services/org.apache.hadoop.hbase.* **/*.proto hbase-webapps/** diff --git a/packaging/hudi-presto-bundle/pom.xml b/packaging/hudi-presto-bundle/pom.xml index 48f483da6667e..51d24987df376 100644 --- a/packaging/hudi-presto-bundle/pom.xml +++ b/packaging/hudi-presto-bundle/pom.xml @@ -61,6 +61,7 @@ META-INF/LICENSE target/classes/META-INF/LICENSE + @@ -154,6 +155,70 @@ org.apache.parquet.avro. ${presto.bundle.bootstrap.shade.prefix}org.apache.parquet.avro. + + org.apache.hadoop.metrics2.MetricHistogram + org.apache.hudi.org.apache.hadoop.metrics2.MetricHistogram + + + + org.apache.hadoop.metrics2.MetricsExecutor + org.apache.hudi.org.apache.hadoop.metrics2.MetricsExecutor + + + + org.apache.hadoop.metrics2.impl.JmxCacheBuster + org.apache.hudi.org.apache.hadoop.metrics2.impl.JmxCacheBuster + + + org.apache.hadoop.metrics2.lib.DefaultMetricsSystemHelper + org.apache.hudi.org.apache.hadoop.metrics2.lib.DefaultMetricsSystemHelper + + + + org.apache.hadoop.metrics2.lib.DynamicMetricsRegistry + org.apache.hudi.org.apache.hadoop.metrics2.lib.DynamicMetricsRegistry + + + + org.apache.hadoop.metrics2.lib.MetricsExecutorImpl + org.apache.hudi.org.apache.hadoop.metrics2.lib.MetricsExecutorImpl + + + + org.apache.hadoop.metrics2.lib.MutableFastCounter + org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableFastCounter + + + + org.apache.hadoop.metrics2.lib.MutableHistogram + org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableHistogram + + + + org.apache.hadoop.metrics2.lib.MutableRangeHistogram + org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableRangeHistogram + + + + org.apache.hadoop.metrics2.lib.MutableSizeHistogram + org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableSizeHistogram + + + + org.apache.hadoop.metrics2.lib.MutableTimeHistogram + org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableTimeHistogram + + + + org.apache.hadoop.metrics2.util.MetricQuantile + org.apache.hudi.org.apache.hadoop.metrics2.util.MetricQuantile + + + + org.apache.hadoop.metrics2.util.MetricSampleQuantiles + org.apache.hudi.org.apache.hadoop.metrics2.util.MetricSampleQuantiles + + false @@ -169,7 +234,6 @@ META-INF/maven/org.apache.hbase.thirdparty/** META-INF/maven/org.apache.htrace/** META-INF/services/javax.* - META-INF/services/org.apache.hadoop.hbase.* **/*.proto com/esotericsoftware/reflectasm/** hbase-webapps/** diff --git a/packaging/hudi-spark-bundle/pom.xml b/packaging/hudi-spark-bundle/pom.xml index 37bc2db5639d0..54eb2a358f0e4 100644 --- a/packaging/hudi-spark-bundle/pom.xml +++ b/packaging/hudi-spark-bundle/pom.xml @@ -63,6 +63,7 @@ META-INF/services/org.apache.spark.sql.sources.DataSourceRegister + @@ -207,6 +208,70 @@ ${spark.bundle.spark.shade.prefix}com.google.common. + + org.apache.hadoop.metrics2.MetricHistogram + org.apache.hudi.org.apache.hadoop.metrics2.MetricHistogram + + + + org.apache.hadoop.metrics2.MetricsExecutor + org.apache.hudi.org.apache.hadoop.metrics2.MetricsExecutor + + + + org.apache.hadoop.metrics2.impl.JmxCacheBuster + org.apache.hudi.org.apache.hadoop.metrics2.impl.JmxCacheBuster + + + org.apache.hadoop.metrics2.lib.DefaultMetricsSystemHelper + org.apache.hudi.org.apache.hadoop.metrics2.lib.DefaultMetricsSystemHelper + + + + org.apache.hadoop.metrics2.lib.DynamicMetricsRegistry + org.apache.hudi.org.apache.hadoop.metrics2.lib.DynamicMetricsRegistry + + + + org.apache.hadoop.metrics2.lib.MetricsExecutorImpl + org.apache.hudi.org.apache.hadoop.metrics2.lib.MetricsExecutorImpl + + + + org.apache.hadoop.metrics2.lib.MutableFastCounter + org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableFastCounter + + + + org.apache.hadoop.metrics2.lib.MutableHistogram + org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableHistogram + + + + org.apache.hadoop.metrics2.lib.MutableRangeHistogram + org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableRangeHistogram + + + + org.apache.hadoop.metrics2.lib.MutableSizeHistogram + org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableSizeHistogram + + + + org.apache.hadoop.metrics2.lib.MutableTimeHistogram + org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableTimeHistogram + + + + org.apache.hadoop.metrics2.util.MetricQuantile + org.apache.hudi.org.apache.hadoop.metrics2.util.MetricQuantile + + + + org.apache.hadoop.metrics2.util.MetricSampleQuantiles + org.apache.hudi.org.apache.hadoop.metrics2.util.MetricSampleQuantiles + + @@ -221,7 +286,6 @@ META-INF/maven/org.apache.hbase.thirdparty/** META-INF/maven/org.apache.htrace/** META-INF/services/javax.* - META-INF/services/org.apache.hadoop.hbase.* **/*.proto hbase-webapps/** diff --git a/packaging/hudi-trino-bundle/pom.xml b/packaging/hudi-trino-bundle/pom.xml index e6cb86c76942c..7deced3a323d9 100644 --- a/packaging/hudi-trino-bundle/pom.xml +++ b/packaging/hudi-trino-bundle/pom.xml @@ -62,6 +62,7 @@ META-INF/LICENSE target/classes/META-INF/LICENSE + @@ -151,6 +152,70 @@ com.google.protobuf. ${trino.bundle.bootstrap.shade.prefix}com.google.protobuf. + + org.apache.hadoop.metrics2.MetricHistogram + org.apache.hudi.org.apache.hadoop.metrics2.MetricHistogram + + + + org.apache.hadoop.metrics2.MetricsExecutor + org.apache.hudi.org.apache.hadoop.metrics2.MetricsExecutor + + + + org.apache.hadoop.metrics2.impl.JmxCacheBuster + org.apache.hudi.org.apache.hadoop.metrics2.impl.JmxCacheBuster + + + org.apache.hadoop.metrics2.lib.DefaultMetricsSystemHelper + org.apache.hudi.org.apache.hadoop.metrics2.lib.DefaultMetricsSystemHelper + + + + org.apache.hadoop.metrics2.lib.DynamicMetricsRegistry + org.apache.hudi.org.apache.hadoop.metrics2.lib.DynamicMetricsRegistry + + + + org.apache.hadoop.metrics2.lib.MetricsExecutorImpl + org.apache.hudi.org.apache.hadoop.metrics2.lib.MetricsExecutorImpl + + + + org.apache.hadoop.metrics2.lib.MutableFastCounter + org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableFastCounter + + + + org.apache.hadoop.metrics2.lib.MutableHistogram + org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableHistogram + + + + org.apache.hadoop.metrics2.lib.MutableRangeHistogram + org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableRangeHistogram + + + + org.apache.hadoop.metrics2.lib.MutableSizeHistogram + org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableSizeHistogram + + + + org.apache.hadoop.metrics2.lib.MutableTimeHistogram + org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableTimeHistogram + + + + org.apache.hadoop.metrics2.util.MetricQuantile + org.apache.hudi.org.apache.hadoop.metrics2.util.MetricQuantile + + + + org.apache.hadoop.metrics2.util.MetricSampleQuantiles + org.apache.hudi.org.apache.hadoop.metrics2.util.MetricSampleQuantiles + + false @@ -166,7 +231,6 @@ META-INF/maven/org.apache.hbase.thirdparty/** META-INF/maven/org.apache.htrace/** META-INF/services/javax.* - META-INF/services/org.apache.hadoop.hbase.* **/*.proto hbase-webapps/** diff --git a/packaging/hudi-utilities-bundle/pom.xml b/packaging/hudi-utilities-bundle/pom.xml index 3c2e34c296d68..5432bc389fac0 100644 --- a/packaging/hudi-utilities-bundle/pom.xml +++ b/packaging/hudi-utilities-bundle/pom.xml @@ -86,6 +86,7 @@ META-INF/services/org.apache.spark.sql.sources.DataSourceRegister + @@ -231,6 +232,70 @@ org.eclipse.jetty. org.apache.hudi.org.eclipse.jetty. + + org.apache.hadoop.metrics2.MetricHistogram + org.apache.hudi.org.apache.hadoop.metrics2.MetricHistogram + + + + org.apache.hadoop.metrics2.MetricsExecutor + org.apache.hudi.org.apache.hadoop.metrics2.MetricsExecutor + + + + org.apache.hadoop.metrics2.impl.JmxCacheBuster + org.apache.hudi.org.apache.hadoop.metrics2.impl.JmxCacheBuster + + + org.apache.hadoop.metrics2.lib.DefaultMetricsSystemHelper + org.apache.hudi.org.apache.hadoop.metrics2.lib.DefaultMetricsSystemHelper + + + + org.apache.hadoop.metrics2.lib.DynamicMetricsRegistry + org.apache.hudi.org.apache.hadoop.metrics2.lib.DynamicMetricsRegistry + + + + org.apache.hadoop.metrics2.lib.MetricsExecutorImpl + org.apache.hudi.org.apache.hadoop.metrics2.lib.MetricsExecutorImpl + + + + org.apache.hadoop.metrics2.lib.MutableFastCounter + org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableFastCounter + + + + org.apache.hadoop.metrics2.lib.MutableHistogram + org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableHistogram + + + + org.apache.hadoop.metrics2.lib.MutableRangeHistogram + org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableRangeHistogram + + + + org.apache.hadoop.metrics2.lib.MutableSizeHistogram + org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableSizeHistogram + + + + org.apache.hadoop.metrics2.lib.MutableTimeHistogram + org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableTimeHistogram + + + + org.apache.hadoop.metrics2.util.MetricQuantile + org.apache.hudi.org.apache.hadoop.metrics2.util.MetricQuantile + + + + org.apache.hadoop.metrics2.util.MetricSampleQuantiles + org.apache.hudi.org.apache.hadoop.metrics2.util.MetricSampleQuantiles + + @@ -245,7 +310,6 @@ META-INF/maven/org.apache.hbase.thirdparty/** META-INF/maven/org.apache.htrace/** META-INF/services/javax.* - META-INF/services/org.apache.hadoop.hbase.* **/*.proto hbase-webapps/** From d8a8b4fbbf0b3051b1274b45f1c00ceb80724050 Mon Sep 17 00:00:00 2001 From: Y Ethan Guo Date: Wed, 2 Mar 2022 15:24:30 -0800 Subject: [PATCH 16/36] Fix hudi-integ-test-bundle --- .../docker-compose_hadoop284_hive233_spark244.yml | 4 ++-- docker/hoodie/hadoop/hive_base/Dockerfile | 2 ++ packaging/hudi-integ-test-bundle/pom.xml | 10 +++++++++- 3 files changed, 13 insertions(+), 3 deletions(-) diff --git a/docker/compose/docker-compose_hadoop284_hive233_spark244.yml b/docker/compose/docker-compose_hadoop284_hive233_spark244.yml index 086004f121e97..572788f2bdfb9 100644 --- a/docker/compose/docker-compose_hadoop284_hive233_spark244.yml +++ b/docker/compose/docker-compose_hadoop284_hive233_spark244.yml @@ -86,7 +86,7 @@ services: container_name: hive-metastore-postgresql hivemetastore: - image: apachehudi/hudi-hadoop_2.8.4-hive_2.3.3:latest + image: yihua/hudi-hadoop_2.8.4-hive_2.3.3:latest hostname: hivemetastore container_name: hivemetastore links: @@ -109,7 +109,7 @@ services: - "namenode" hiveserver: - image: apachehudi/hudi-hadoop_2.8.4-hive_2.3.3:latest + image: yihua/hudi-hadoop_2.8.4-hive_2.3.3:latest hostname: hiveserver container_name: hiveserver env_file: diff --git a/docker/hoodie/hadoop/hive_base/Dockerfile b/docker/hoodie/hadoop/hive_base/Dockerfile index 7d04d94fc60cc..cf40bdcadca8f 100644 --- a/docker/hoodie/hadoop/hive_base/Dockerfile +++ b/docker/hoodie/hadoop/hive_base/Dockerfile @@ -36,6 +36,8 @@ RUN echo "Hive URL is :${HIVE_URL}" && wget ${HIVE_URL} -O hive.tar.gz && \ wget https://jdbc.postgresql.org/download/postgresql-9.4.1212.jar -O $HIVE_HOME/lib/postgresql-jdbc.jar && \ rm hive.tar.gz && mkdir -p /var/hoodie/ws/docker/hoodie/hadoop/hive_base/target/ +RUN rm hive/lib/hbase* + #Spark should be compiled with Hive to be able to use it #hive-site.xml should be copied to $SPARK_HOME/conf folder diff --git a/packaging/hudi-integ-test-bundle/pom.xml b/packaging/hudi-integ-test-bundle/pom.xml index b53e02aaf7768..e11212a4a34a7 100644 --- a/packaging/hudi-integ-test-bundle/pom.xml +++ b/packaging/hudi-integ-test-bundle/pom.xml @@ -62,6 +62,9 @@ META-INF/services/org.apache.spark.sql.sources.DataSourceRegister + + hbase-default.xml + @@ -133,7 +136,6 @@ org.apache.hive:hive-common org.apache.hive:hive-service - org.apache.hive:hive-metastore org.apache.hive:hive-jdbc org.apache.hive:hive-exec @@ -398,6 +400,12 @@ hive-metastore ${hive.version} provided + + + org.apache.hbase + * + + From cf882fb8ca3f279fe2841f6ae99d77e834e8c339 Mon Sep 17 00:00:00 2001 From: Y Ethan Guo Date: Wed, 2 Mar 2022 15:56:27 -0800 Subject: [PATCH 17/36] Fix rebase issue --- .../apache/hudi/common/table/log/block/HoodieHFileDataBlock.java | 1 - 1 file changed, 1 deletion(-) diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieHFileDataBlock.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieHFileDataBlock.java index d54582233abfd..e843ad74cb31c 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieHFileDataBlock.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieHFileDataBlock.java @@ -41,7 +41,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.hbase.util.Pair; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; From a9f41b29da5e2a62a720cef347aa14a8653cd8f5 Mon Sep 17 00:00:00 2001 From: Y Ethan Guo Date: Fri, 4 Mar 2022 12:04:56 -0800 Subject: [PATCH 18/36] Add hbase-site to skip version check --- ...ker-compose_hadoop284_hive233_spark244.yml | 4 +-- hudi-hadoop-mr/pom.xml | 3 ++ .../src/main/resources/hbase-site.xml | 36 +++++++++++++++++++ 3 files changed, 41 insertions(+), 2 deletions(-) create mode 100644 hudi-hadoop-mr/src/main/resources/hbase-site.xml diff --git a/docker/compose/docker-compose_hadoop284_hive233_spark244.yml b/docker/compose/docker-compose_hadoop284_hive233_spark244.yml index 572788f2bdfb9..086004f121e97 100644 --- a/docker/compose/docker-compose_hadoop284_hive233_spark244.yml +++ b/docker/compose/docker-compose_hadoop284_hive233_spark244.yml @@ -86,7 +86,7 @@ services: container_name: hive-metastore-postgresql hivemetastore: - image: yihua/hudi-hadoop_2.8.4-hive_2.3.3:latest + image: apachehudi/hudi-hadoop_2.8.4-hive_2.3.3:latest hostname: hivemetastore container_name: hivemetastore links: @@ -109,7 +109,7 @@ services: - "namenode" hiveserver: - image: yihua/hudi-hadoop_2.8.4-hive_2.3.3:latest + image: apachehudi/hudi-hadoop_2.8.4-hive_2.3.3:latest hostname: hiveserver container_name: hiveserver env_file: diff --git a/hudi-hadoop-mr/pom.xml b/hudi-hadoop-mr/pom.xml index bf87bfaa36a81..8aa378036d9c0 100644 --- a/hudi-hadoop-mr/pom.xml +++ b/hudi-hadoop-mr/pom.xml @@ -131,6 +131,9 @@ src/main/resources + + hbase-site.xml + diff --git a/hudi-hadoop-mr/src/main/resources/hbase-site.xml b/hudi-hadoop-mr/src/main/resources/hbase-site.xml new file mode 100644 index 0000000000000..00b093727e3c5 --- /dev/null +++ b/hudi-hadoop-mr/src/main/resources/hbase-site.xml @@ -0,0 +1,36 @@ + + + + + + hbase.defaults.for.version.skip + true + Set to true to skip the 'hbase.defaults.for.version' check. + Setting this to true can be useful in contexts other than + the other side of a maven generation; i.e. running in an + IDE. You'll want to set this boolean to true to avoid + seeing the RuntimeException complaint: "hbase-default.xml file + seems to be for and old version of HBase (\${hbase.version}), this + version is X.X.X-SNAPSHOT" + + + \ No newline at end of file From feb4cb44451bd400efedcdf1645687cb4cff6685 Mon Sep 17 00:00:00 2001 From: Y Ethan Guo Date: Tue, 8 Mar 2022 18:24:27 -0800 Subject: [PATCH 19/36] Add hbase-site.xml to hudi-common --- hudi-common/src/main/resources/hbase-site.xml | 2185 +++++++++++++++++ hudi-hadoop-mr/pom.xml | 8 - .../src/main/resources/hbase-site.xml | 36 - packaging/hudi-hadoop-mr-bundle/pom.xml | 7 +- 4 files changed, 2186 insertions(+), 50 deletions(-) create mode 100644 hudi-common/src/main/resources/hbase-site.xml delete mode 100644 hudi-hadoop-mr/src/main/resources/hbase-site.xml diff --git a/hudi-common/src/main/resources/hbase-site.xml b/hudi-common/src/main/resources/hbase-site.xml new file mode 100644 index 0000000000000..ad680e6b8999e --- /dev/null +++ b/hudi-common/src/main/resources/hbase-site.xml @@ -0,0 +1,2185 @@ + + + + + + + + + + + + hbase.tmp.dir + ${java.io.tmpdir}/hbase-${user.name} + Temporary directory on the local filesystem. + Change this setting to point to a location more permanent + than '/tmp', the usual resolve for java.io.tmpdir, as the + '/tmp' directory is cleared on machine restart. + + + + hbase.rootdir + ${hbase.tmp.dir}/hbase + The directory shared by region servers and into + which HBase persists. The URL should be 'fully-qualified' + to include the filesystem scheme. For example, to specify the + HDFS directory '/hbase' where the HDFS instance's namenode is + running at namenode.example.org on port 9000, set this value to: + hdfs://namenode.example.org:9000/hbase. By default, we write + to whatever ${hbase.tmp.dir} is set too -- usually /tmp -- + so change this configuration or else all data will be lost on + machine restart. + + + + hbase.cluster.distributed + false + The mode the cluster will be in. Possible values are + false for standalone mode and true for distributed mode. If + false, startup will run all HBase and ZooKeeper daemons together + in the one JVM. + + + + hbase.zookeeper.quorum + + 127.0.0.1 + Comma separated list of servers in the ZooKeeper ensemble + (This config. should have been named hbase.zookeeper.ensemble). + For example, "host1.mydomain.com,host2.mydomain.com,host3.mydomain.com". + By default this is set to localhost for local and pseudo-distributed modes + of operation. For a fully-distributed setup, this should be set to a full + list of ZooKeeper ensemble servers. If HBASE_MANAGES_ZK is set in hbase-env.sh + this is the list of servers which hbase will start/stop ZooKeeper on as + part of cluster start/stop. Client-side, we will take this list of + ensemble members and put it together with the hbase.zookeeper.property.clientPort + config. and pass it into zookeeper constructor as the connectString + parameter. + + + + + + zookeeper.recovery.retry.maxsleeptime + 60000 + Max sleep time before retry zookeeper operations in milliseconds, + a max time is needed here so that sleep time won't grow unboundedly + + + + hbase.local.dir + ${hbase.tmp.dir}/local/ + Directory on the local filesystem to be used + as a local storage. + + + + + + hbase.master.port + 16000 + The port the HBase Master should bind to. + + + hbase.master.info.port + 16010 + The port for the HBase Master web UI. + Set to -1 if you do not want a UI instance run. + + + + hbase.master.info.bindAddress + 0.0.0.0 + The bind address for the HBase Master web UI + + + + hbase.master.logcleaner.plugins + + org.apache.hadoop.hbase.master.cleaner.TimeToLiveLogCleaner,org.apache.hadoop.hbase.master.cleaner.TimeToLiveProcedureWALCleaner,org.apache.hadoop.hbase.master.cleaner.TimeToLiveMasterLocalStoreWALCleaner + + A comma-separated list of BaseLogCleanerDelegate invoked by + the LogsCleaner service. These WAL cleaners are called in order, + so put the cleaner that prunes the most files in front. To + implement your own BaseLogCleanerDelegate, just put it in HBase's classpath + and add the fully qualified class name here. Always add the above + default log cleaners in the list. + + + + hbase.master.logcleaner.ttl + 600000 + How long a WAL remain in the archive ({hbase.rootdir}/oldWALs) directory, + after which it will be cleaned by a Master thread. The value is in milliseconds. + + + + hbase.master.hfilecleaner.plugins + + org.apache.hadoop.hbase.master.cleaner.TimeToLiveHFileCleaner,org.apache.hadoop.hbase.master.cleaner.TimeToLiveMasterLocalStoreHFileCleaner + + A comma-separated list of BaseHFileCleanerDelegate invoked by + the HFileCleaner service. These HFiles cleaners are called in order, + so put the cleaner that prunes the most files in front. To + implement your own BaseHFileCleanerDelegate, just put it in HBase's classpath + and add the fully qualified class name here. Always add the above + default hfile cleaners in the list as they will be overwritten in + hbase-site.xml. + + + + hbase.master.infoserver.redirect + true + Whether or not the Master listens to the Master web + UI port (hbase.master.info.port) and redirects requests to the web + UI server shared by the Master and RegionServer. Config. makes + sense when Master is serving Regions (not the default). + + + + hbase.master.fileSplitTimeout + 600000 + Splitting a region, how long to wait on the file-splitting + step before aborting the attempt. Default: 600000. This setting used + to be known as hbase.regionserver.fileSplitTimeout in hbase-1.x. + Split is now run master-side hence the rename (If a + 'hbase.master.fileSplitTimeout' setting found, will use it to + prime the current 'hbase.master.fileSplitTimeout' + Configuration. + + + + + + hbase.regionserver.port + 16020 + The port the HBase RegionServer binds to. + + + hbase.regionserver.info.port + 16030 + The port for the HBase RegionServer web UI + Set to -1 if you do not want the RegionServer UI to run. + + + + hbase.regionserver.info.bindAddress + 0.0.0.0 + The address for the HBase RegionServer web UI + + + hbase.regionserver.info.port.auto + false + Whether or not the Master or RegionServer + UI should search for a port to bind to. Enables automatic port + search if hbase.regionserver.info.port is already in use. + Useful for testing, turned off by default. + + + + hbase.regionserver.handler.count + 30 + Count of RPC Listener instances spun up on RegionServers. + Same property is used by the Master for count of master handlers. + Too many handlers can be counter-productive. Make it a multiple of + CPU count. If mostly read-only, handlers count close to cpu count + does well. Start with twice the CPU count and tune from there. + + + + hbase.ipc.server.callqueue.handler.factor + 0.1 + Factor to determine the number of call queues. + A value of 0 means a single queue shared between all the handlers. + A value of 1 means that each handler has its own queue. + + + + hbase.ipc.server.callqueue.read.ratio + 0 + Split the call queues into read and write queues. + The specified interval (which should be between 0.0 and 1.0) + will be multiplied by the number of call queues. + A value of 0 indicate to not split the call queues, meaning that both read and write + requests will be pushed to the same set of queues. + A value lower than 0.5 means that there will be less read queues than write queues. + A value of 0.5 means there will be the same number of read and write queues. + A value greater than 0.5 means that there will be more read queues than write queues. + A value of 1.0 means that all the queues except one are used to dispatch read requests. + + Example: Given the total number of call queues being 10 + a read.ratio of 0 means that: the 10 queues will contain both read/write requests. + a read.ratio of 0.3 means that: 3 queues will contain only read requests + and 7 queues will contain only write requests. + a read.ratio of 0.5 means that: 5 queues will contain only read requests + and 5 queues will contain only write requests. + a read.ratio of 0.8 means that: 8 queues will contain only read requests + and 2 queues will contain only write requests. + a read.ratio of 1 means that: 9 queues will contain only read requests + and 1 queues will contain only write requests. + + + + hbase.ipc.server.callqueue.scan.ratio + 0 + Given the number of read call queues, calculated from the total number + of call queues multiplied by the callqueue.read.ratio, the scan.ratio property + will split the read call queues into small-read and long-read queues. + A value lower than 0.5 means that there will be less long-read queues than short-read queues. + A value of 0.5 means that there will be the same number of short-read and long-read queues. + A value greater than 0.5 means that there will be more long-read queues than short-read queues + A value of 0 or 1 indicate to use the same set of queues for gets and scans. + + Example: Given the total number of read call queues being 8 + a scan.ratio of 0 or 1 means that: 8 queues will contain both long and short read requests. + a scan.ratio of 0.3 means that: 2 queues will contain only long-read requests + and 6 queues will contain only short-read requests. + a scan.ratio of 0.5 means that: 4 queues will contain only long-read requests + and 4 queues will contain only short-read requests. + a scan.ratio of 0.8 means that: 6 queues will contain only long-read requests + and 2 queues will contain only short-read requests. + + + + hbase.regionserver.msginterval + 3000 + Interval between messages from the RegionServer to Master + in milliseconds. + + + + hbase.regionserver.logroll.period + 3600000 + Period at which we will roll the commit log regardless + of how many edits it has. + + + + hbase.regionserver.logroll.errors.tolerated + 2 + The number of consecutive WAL close errors we will allow + before triggering a server abort. A setting of 0 will cause the + region server to abort if closing the current WAL writer fails during + log rolling. Even a small value (2 or 3) will allow a region server + to ride over transient HDFS errors. + + + + hbase.regionserver.hlog.reader.impl + org.apache.hadoop.hbase.regionserver.wal.ProtobufLogReader + The WAL file reader implementation. + + + hbase.regionserver.hlog.writer.impl + org.apache.hadoop.hbase.regionserver.wal.ProtobufLogWriter + The WAL file writer implementation. + + + hbase.regionserver.global.memstore.size + + Maximum size of all memstores in a region server before new + updates are blocked and flushes are forced. Defaults to 40% of heap (0.4). + Updates are blocked and flushes are forced until size of all memstores + in a region server hits hbase.regionserver.global.memstore.size.lower.limit. + The default value in this configuration has been intentionally left empty in order to + honor the old hbase.regionserver.global.memstore.upperLimit property if present. + + + + hbase.regionserver.global.memstore.size.lower.limit + + Maximum size of all memstores in a region server before flushes + are forced. Defaults to 95% of hbase.regionserver.global.memstore.size + (0.95). A 100% value for this value causes the minimum possible flushing + to occur when updates are blocked due to memstore limiting. The default + value in this configuration has been intentionally left empty in order to + honor the old hbase.regionserver.global.memstore.lowerLimit property if + present. + + + + hbase.systemtables.compacting.memstore.type + NONE + Determines the type of memstore to be used for system tables like + META, namespace tables etc. By default NONE is the type and hence we use the + default memstore for all the system tables. If we need to use compacting + memstore for system tables then set this property to BASIC/EAGER + + + + hbase.regionserver.optionalcacheflushinterval + 3600000 + + Maximum amount of time an edit lives in memory before being automatically flushed. + Default 1 hour. Set it to 0 to disable automatic flushing. + + + + hbase.regionserver.dns.interface + default + The name of the Network Interface from which a region server + should report its IP address. + + + + hbase.regionserver.dns.nameserver + default + The host name or IP address of the name server (DNS) + which a region server should use to determine the host name used by the + master for communication and display purposes. + + + + hbase.regionserver.region.split.policy + org.apache.hadoop.hbase.regionserver.SteppingSplitPolicy + + A split policy determines when a region should be split. The various + other split policies that are available currently are BusyRegionSplitPolicy, + ConstantSizeRegionSplitPolicy, DisabledRegionSplitPolicy, + DelimitedKeyPrefixRegionSplitPolicy, KeyPrefixRegionSplitPolicy, and + SteppingSplitPolicy. DisabledRegionSplitPolicy blocks manual region splitting. + + + + hbase.regionserver.regionSplitLimit + 1000 + + Limit for the number of regions after which no more region splitting + should take place. This is not hard limit for the number of regions + but acts as a guideline for the regionserver to stop splitting after + a certain limit. Default is set to 1000. + + + + + + zookeeper.session.timeout + 90000 + ZooKeeper session timeout in milliseconds. It is used in two different ways. + First, this value is used in the ZK client that HBase uses to connect to the ensemble. + It is also used by HBase when it starts a ZK server and it is passed as the 'maxSessionTimeout'. + See https://zookeeper.apache.org/doc/current/zookeeperProgrammers.html#ch_zkSessions. + For example, if an HBase region server connects to a ZK ensemble that's also managed + by HBase, then the session timeout will be the one specified by this configuration. + But, a region server that connects to an ensemble managed with a different configuration + will be subjected that ensemble's maxSessionTimeout. So, even though HBase might propose + using 90 seconds, the ensemble can have a max timeout lower than this and it will take + precedence. The current default maxSessionTimeout that ZK ships with is 40 seconds, which is lower than + HBase's. + + + + zookeeper.znode.parent + /hbase + Root ZNode for HBase in ZooKeeper. All of HBase's ZooKeeper + files that are configured with a relative path will go under this node. + By default, all of HBase's ZooKeeper file paths are configured with a + relative path, so they will all go under this directory unless changed. + + + + zookeeper.znode.acl.parent + acl + Root ZNode for access control lists. + + + hbase.zookeeper.dns.interface + default + The name of the Network Interface from which a ZooKeeper server + should report its IP address. + + + + hbase.zookeeper.dns.nameserver + default + The host name or IP address of the name server (DNS) + which a ZooKeeper server should use to determine the host name used by the + master for communication and display purposes. + + + + + hbase.zookeeper.peerport + 2888 + Port used by ZooKeeper peers to talk to each other. + See https://zookeeper.apache.org/doc/r3.3.3/zookeeperStarted.html#sc_RunningReplicatedZooKeeper + for more information. + + + + hbase.zookeeper.leaderport + 3888 + Port used by ZooKeeper for leader election. + See https://zookeeper.apache.org/doc/r3.3.3/zookeeperStarted.html#sc_RunningReplicatedZooKeeper + for more information. + + + + + + + hbase.zookeeper.property.initLimit + 10 + Property from ZooKeeper's config zoo.cfg. + The number of ticks that the initial synchronization phase can take. + + + + hbase.zookeeper.property.syncLimit + 5 + Property from ZooKeeper's config zoo.cfg. + The number of ticks that can pass between sending a request and getting an + acknowledgment. + + + + hbase.zookeeper.property.dataDir + ${hbase.tmp.dir}/zookeeper + Property from ZooKeeper's config zoo.cfg. + The directory where the snapshot is stored. + + + + hbase.zookeeper.property.clientPort + 2181 + Property from ZooKeeper's config zoo.cfg. + The port at which the clients will connect. + + + + hbase.zookeeper.property.maxClientCnxns + 300 + Property from ZooKeeper's config zoo.cfg. + Limit on number of concurrent connections (at the socket level) that a + single client, identified by IP address, may make to a single member of + the ZooKeeper ensemble. Set high to avoid zk connection issues running + standalone and pseudo-distributed. + + + + + + + hbase.client.write.buffer + 2097152 + Default size of the BufferedMutator write buffer in bytes. + A bigger buffer takes more memory -- on both the client and server + side since server instantiates the passed write buffer to process + it -- but a larger buffer size reduces the number of RPCs made. + For an estimate of server-side memory-used, evaluate + hbase.client.write.buffer * hbase.regionserver.handler.count + + + + hbase.client.pause + 100 + General client pause value. Used mostly as value to wait + before running a retry of a failed get, region lookup, etc. + See hbase.client.retries.number for description of how we backoff from + this initial pause amount and how this pause works w/ retries. + + + + hbase.client.pause.cqtbe + + Whether or not to use a special client pause for + CallQueueTooBigException (cqtbe). Set this property to a higher value + than hbase.client.pause if you observe frequent CQTBE from the same + RegionServer and the call queue there keeps full + + + + hbase.client.retries.number + 15 + Maximum retries. Used as maximum for all retryable + operations such as the getting of a cell's value, starting a row update, + etc. Retry interval is a rough function based on hbase.client.pause. At + first we retry at this interval but then with backoff, we pretty quickly reach + retrying every ten seconds. See HConstants#RETRY_BACKOFF for how the backup + ramps up. Change this setting and hbase.client.pause to suit your workload. + + + + hbase.client.max.total.tasks + 100 + The maximum number of concurrent mutation tasks a single HTable instance will + send to the cluster. + + + + hbase.client.max.perserver.tasks + 2 + The maximum number of concurrent mutation tasks a single HTable instance will + send to a single region server. + + + + hbase.client.max.perregion.tasks + 1 + The maximum number of concurrent mutation tasks the client will + maintain to a single Region. That is, if there is already + hbase.client.max.perregion.tasks writes in progress for this region, new puts + won't be sent to this region until some writes finishes. + + + + hbase.client.perserver.requests.threshold + 2147483647 + The max number of concurrent pending requests for one server in all client threads + (process level). Exceeding requests will be thrown ServerTooBusyException immediately to prevent + user's threads being occupied and blocked by only one slow region server. If you use a fix + number of threads to access HBase in a synchronous way, set this to a suitable value which is + related to the number of threads will help you. See + https://issues.apache.org/jira/browse/HBASE-16388 for details. + + + + hbase.client.scanner.caching + 2147483647 + Number of rows that we try to fetch when calling next + on a scanner if it is not served from (local, client) memory. This configuration + works together with hbase.client.scanner.max.result.size to try and use the + network efficiently. The default value is Integer.MAX_VALUE by default so that + the network will fill the chunk size defined by hbase.client.scanner.max.result.size + rather than be limited by a particular number of rows since the size of rows varies + table to table. If you know ahead of time that you will not require more than a certain + number of rows from a scan, this configuration should be set to that row limit via + Scan#setCaching. Higher caching values will enable faster scanners but will eat up more + memory and some calls of next may take longer and longer times when the cache is empty. + Do not set this value such that the time between invocations is greater than the scanner + timeout; i.e. hbase.client.scanner.timeout.period + + + + hbase.client.keyvalue.maxsize + 10485760 + Specifies the combined maximum allowed size of a KeyValue + instance. This is to set an upper boundary for a single entry saved in a + storage file. Since they cannot be split it helps avoiding that a region + cannot be split any further because the data is too large. It seems wise + to set this to a fraction of the maximum region size. Setting it to zero + or less disables the check. + + + + hbase.server.keyvalue.maxsize + 10485760 + Maximum allowed size of an individual cell, inclusive of value and all key + components. A value of 0 or less disables the check. + The default value is 10MB. + This is a safety setting to protect the server from OOM situations. + + + + hbase.client.scanner.timeout.period + 60000 + Client scanner lease period in milliseconds. + + + hbase.client.localityCheck.threadPoolSize + 2 + + + + + hbase.bulkload.retries.number + 10 + Maximum retries. This is maximum number of iterations + to atomic bulk loads are attempted in the face of splitting operations + 0 means never give up. + + + + hbase.master.balancer.maxRitPercent + 1.0 + The max percent of regions in transition when balancing. + The default value is 1.0. So there are no balancer throttling. If set this config to 0.01, + It means that there are at most 1% regions in transition when balancing. + Then the cluster's availability is at least 99% when balancing. + + + + hbase.balancer.period + + 300000 + Period at which the region balancer runs in the Master, in + milliseconds. + + + + hbase.regions.slop + 0.001 + Rebalance if any regionserver has average + (average * slop) regions. + The default value of this parameter is 0.001 in StochasticLoadBalancer (the default load + balancer), while the default is 0.2 in other load balancers (i.e., + SimpleLoadBalancer). + + + + hbase.normalizer.period + 300000 + Period at which the region normalizer runs in the Master, in + milliseconds. + + + + hbase.normalizer.split.enabled + true + Whether to split a region as part of normalization. + + + hbase.normalizer.merge.enabled + true + Whether to merge a region as part of normalization. + + + hbase.normalizer.min.region.count + 3 + The minimum number of regions in a table to consider it for merge + normalization. + + + + hbase.normalizer.merge.min_region_age.days + 3 + The minimum age for a region to be considered for a merge, in days. + + + hbase.normalizer.merge.min_region_age.days + 3 + The minimum age for a region to be considered for a merge, in days. + + + hbase.normalizer.merge.min_region_size.mb + 1 + The minimum size for a region to be considered for a merge, in whole + MBs. + + + + hbase.table.normalization.enabled + false + This config is used to set default behaviour of normalizer at table level. + To override this at table level one can set NORMALIZATION_ENABLED at table descriptor level + and that property will be honored + + + + hbase.server.thread.wakefrequency + 10000 + Time to sleep in between searches for work (in milliseconds). + Used as sleep interval by service threads such as log roller. + + + + hbase.server.versionfile.writeattempts + 3 + + How many times to retry attempting to write a version file + before just aborting. Each attempt is separated by the + hbase.server.thread.wakefrequency milliseconds. + + + + hbase.hregion.memstore.flush.size + 134217728 + + Memstore will be flushed to disk if size of the memstore + exceeds this number of bytes. Value is checked by a thread that runs + every hbase.server.thread.wakefrequency. + + + + hbase.hregion.percolumnfamilyflush.size.lower.bound.min + 16777216 + + If FlushLargeStoresPolicy is used and there are multiple column families, + then every time that we hit the total memstore limit, we find out all the + column families whose memstores exceed a "lower bound" and only flush them + while retaining the others in memory. The "lower bound" will be + "hbase.hregion.memstore.flush.size / column_family_number" by default + unless value of this property is larger than that. If none of the families + have their memstore size more than lower bound, all the memstores will be + flushed (just as usual). + + + + hbase.hregion.preclose.flush.size + 5242880 + + If the memstores in a region are this size or larger when we go + to close, run a "pre-flush" to clear out memstores before we put up + the region closed flag and take the region offline. On close, + a flush is run under the close flag to empty memory. During + this time the region is offline and we are not taking on any writes. + If the memstore content is large, this flush could take a long time to + complete. The preflush is meant to clean out the bulk of the memstore + before putting up the close flag and taking the region offline so the + flush that runs under the close flag has little to do. + + + + hbase.hregion.memstore.block.multiplier + 4 + + Block updates if memstore has hbase.hregion.memstore.block.multiplier + times hbase.hregion.memstore.flush.size bytes. Useful preventing + runaway memstore during spikes in update traffic. Without an + upper-bound, memstore fills such that when it flushes the + resultant flush files take a long time to compact or split, or + worse, we OOME. + + + + hbase.hregion.memstore.mslab.enabled + true + + Enables the MemStore-Local Allocation Buffer, + a feature which works to prevent heap fragmentation under + heavy write loads. This can reduce the frequency of stop-the-world + GC pauses on large heaps. + + + + hbase.hregion.memstore.mslab.chunksize + 2097152 + The maximum byte size of a chunk in the MemStoreLAB. Unit: bytes + + + hbase.regionserver.offheap.global.memstore.size + 0 + The amount of off-heap memory all MemStores in a RegionServer may use. + A value of 0 means that no off-heap memory will be used and all chunks in MSLAB + will be HeapByteBuffer, otherwise the non-zero value means how many megabyte of + off-heap memory will be used for chunks in MSLAB and all chunks in MSLAB will be + DirectByteBuffer. Unit: megabytes. + + + + hbase.hregion.memstore.mslab.max.allocation + 262144 + The maximal size of one allocation in the MemStoreLAB, if the desired byte + size exceed this threshold then it will be just allocated from JVM heap rather than MemStoreLAB. + + + + hbase.hregion.max.filesize + 10737418240 + + Maximum HFile size. If the sum of the sizes of a region's HFiles has grown to exceed this + value, the region is split in two. + + + + hbase.hregion.split.overallfiles + false + If we should sum overall region files size when check to split. + + + hbase.hregion.majorcompaction + 604800000 + Time between major compactions, expressed in milliseconds. Set to 0 to disable + time-based automatic major compactions. User-requested and size-based major compactions will + still run. This value is multiplied by hbase.hregion.majorcompaction.jitter to cause + compaction to start at a somewhat-random time during a given window of time. The default value + is 7 days, expressed in milliseconds. If major compactions are causing disruption in your + environment, you can configure them to run at off-peak times for your deployment, or disable + time-based major compactions by setting this parameter to 0, and run major compactions in a + cron job or by another external mechanism. + + + + hbase.hregion.majorcompaction.jitter + 0.50 + A multiplier applied to hbase.hregion.majorcompaction to cause compaction to occur + a given amount of time either side of hbase.hregion.majorcompaction. The smaller the number, + the closer the compactions will happen to the hbase.hregion.majorcompaction + interval. + + + + hbase.hstore.compactionThreshold + 3 + If more than this number of StoreFiles exist in any one Store + (one StoreFile is written per flush of MemStore), a compaction is run to rewrite all + StoreFiles into a single StoreFile. Larger values delay compaction, but when compaction does + occur, it takes longer to complete. + + + + hbase.regionserver.compaction.enabled + true + Enable/disable compactions on by setting true/false. + We can further switch compactions dynamically with the + compaction_switch shell command. + + + + hbase.hstore.flusher.count + 2 + The number of flush threads. With fewer threads, the MemStore flushes will be + queued. With more threads, the flushes will be executed in parallel, increasing the load on + HDFS, and potentially causing more compactions. + + + + hbase.hstore.blockingStoreFiles + 16 + If more than this number of StoreFiles exist in any one Store (one StoreFile + is written per flush of MemStore), updates are blocked for this region until a compaction is + completed, or until hbase.hstore.blockingWaitTime has been exceeded. + + + + hbase.hstore.blockingWaitTime + 90000 + The time for which a region will block updates after reaching the StoreFile limit + defined by hbase.hstore.blockingStoreFiles. After this time has elapsed, the region will stop + blocking updates even if a compaction has not been completed. + + + + hbase.hstore.compaction.min + + The minimum number of StoreFiles which must be eligible for compaction before + compaction can run. The goal of tuning hbase.hstore.compaction.min is to avoid ending up with + too many tiny StoreFiles to compact. Setting this value to 2 would cause a minor compaction + each time you have two StoreFiles in a Store, and this is probably not appropriate. If you + set this value too high, all the other values will need to be adjusted accordingly. For most + cases, the default value is appropriate (empty value here, results in 3 by code logic). In + previous versions of HBase, the parameter hbase.hstore.compaction.min was named + hbase.hstore.compactionThreshold. + + + + hbase.hstore.compaction.max + 10 + The maximum number of StoreFiles which will be selected for a single minor + compaction, regardless of the number of eligible StoreFiles. Effectively, the value of + hbase.hstore.compaction.max controls the length of time it takes a single compaction to + complete. Setting it larger means that more StoreFiles are included in a compaction. For most + cases, the default value is appropriate. + + + + hbase.hstore.compaction.min.size + 134217728 + A StoreFile (or a selection of StoreFiles, when using ExploringCompactionPolicy) + smaller than this size will always be eligible for minor compaction. + HFiles this size or larger are evaluated by hbase.hstore.compaction.ratio to determine if + they are eligible. Because this limit represents the "automatic include" limit for all + StoreFiles smaller than this value, this value may need to be reduced in write-heavy + environments where many StoreFiles in the 1-2 MB range are being flushed, because every + StoreFile will be targeted for compaction and the resulting StoreFiles may still be under the + minimum size and require further compaction. If this parameter is lowered, the ratio check is + triggered more quickly. This addressed some issues seen in earlier versions of HBase but + changing this parameter is no longer necessary in most situations. Default: 128 MB expressed + in bytes. + + + + hbase.hstore.compaction.max.size + 9223372036854775807 + A StoreFile (or a selection of StoreFiles, when using ExploringCompactionPolicy) + larger than this size will be excluded from compaction. The effect of + raising hbase.hstore.compaction.max.size is fewer, larger StoreFiles that do not get + compacted often. If you feel that compaction is happening too often without much benefit, you + can try raising this value. Default: the value of LONG.MAX_VALUE, expressed in bytes. + + + + hbase.hstore.compaction.ratio + 1.2F + For minor compaction, this ratio is used to determine whether a given StoreFile + which is larger than hbase.hstore.compaction.min.size is eligible for compaction. Its + effect is to limit compaction of large StoreFiles. The value of hbase.hstore.compaction.ratio + is expressed as a floating-point decimal. A large ratio, such as 10, will produce a single + giant StoreFile. Conversely, a low value, such as .25, will produce behavior similar to the + BigTable compaction algorithm, producing four StoreFiles. A moderate value of between 1.0 and + 1.4 is recommended. When tuning this value, you are balancing write costs with read costs. + Raising the value (to something like 1.4) will have more write costs, because you will + compact larger StoreFiles. However, during reads, HBase will need to seek through fewer + StoreFiles to accomplish the read. Consider this approach if you cannot take advantage of + Bloom filters. Otherwise, you can lower this value to something like 1.0 to reduce the + background cost of writes, and use Bloom filters to control the number of StoreFiles touched + during reads. For most cases, the default value is appropriate. + + + + hbase.hstore.compaction.ratio.offpeak + 5.0F + Allows you to set a different (by default, more aggressive) ratio for determining + whether larger StoreFiles are included in compactions during off-peak hours. Works in the + same way as hbase.hstore.compaction.ratio. Only applies if hbase.offpeak.start.hour and + hbase.offpeak.end.hour are also enabled. + + + + hbase.hstore.time.to.purge.deletes + 0 + The amount of time to delay purging of delete markers with future timestamps. If + unset, or set to 0, all delete markers, including those with future timestamps, are purged + during the next major compaction. Otherwise, a delete marker is kept until the major compaction + which occurs after the marker's timestamp plus the value of this setting, in milliseconds. + + + + hbase.offpeak.start.hour + -1 + The start of off-peak hours, expressed as an integer between 0 and 23, inclusive. + Set to -1 to disable off-peak. + + + + hbase.offpeak.end.hour + -1 + The end of off-peak hours, expressed as an integer between 0 and 23, inclusive. Set + to -1 to disable off-peak. + + + + hbase.regionserver.thread.compaction.throttle + 2684354560 + There are two different thread pools for compactions, one for large compactions and + the other for small compactions. This helps to keep compaction of lean tables (such as + hbase:meta) fast. If a compaction is larger than this threshold, it + goes into the large compaction pool. In most cases, the default value is appropriate. Default: + 2 x hbase.hstore.compaction.max x hbase.hregion.memstore.flush.size (which defaults to 128MB). + The value field assumes that the value of hbase.hregion.memstore.flush.size is unchanged from + the default. + + + + hbase.regionserver.majorcompaction.pagecache.drop + true + Specifies whether to drop pages read/written into the system page cache by + major compactions. Setting it to true helps prevent major compactions from + polluting the page cache, which is almost always required, especially for clusters + with low/moderate memory to storage ratio. + + + + hbase.regionserver.minorcompaction.pagecache.drop + true + Specifies whether to drop pages read/written into the system page cache by + minor compactions. Setting it to true helps prevent minor compactions from + polluting the page cache, which is most beneficial on clusters with low + memory to storage ratio or very write heavy clusters. You may want to set it to + false under moderate to low write workload when bulk of the reads are + on the most recently written data. + + + + hbase.hstore.compaction.kv.max + 10 + The maximum number of KeyValues to read and then write in a batch when flushing or + compacting. Set this lower if you have big KeyValues and problems with Out Of Memory + Exceptions Set this higher if you have wide, small rows. + + + + hbase.storescanner.parallel.seek.enable + false + + Enables StoreFileScanner parallel-seeking in StoreScanner, + a feature which can reduce response latency under special conditions. + + + + hbase.storescanner.parallel.seek.threads + 10 + + The default thread pool size if parallel-seeking feature enabled. + + + + hfile.block.cache.policy + LRU + The eviction policy for the L1 block cache (LRU or TinyLFU). + + + hfile.block.cache.size + 0.4 + Percentage of maximum heap (-Xmx setting) to allocate to block cache + used by a StoreFile. Default of 0.4 means allocate 40%. + Set to 0 to disable but it's not recommended; you need at least + enough cache to hold the storefile indices. + + + + hfile.block.index.cacheonwrite + false + This allows to put non-root multi-level index blocks into the block + cache at the time the index is being written. + + + + hfile.index.block.max.size + 131072 + When the size of a leaf-level, intermediate-level, or root-level + index block in a multi-level block index grows to this size, the + block is written out and a new block is started. + + + + hbase.bucketcache.ioengine + + Where to store the contents of the bucketcache. One of: offheap, + file, files, mmap or pmem. If a file or files, set it to file(s):PATH_TO_FILE. + mmap means the content will be in an mmaped file. Use mmap:PATH_TO_FILE. 'pmem' + is bucket cache over a file on the persistent memory device. + Use pmem:PATH_TO_FILE. + See http://hbase.apache.org/book.html#offheap.blockcache for more information. + + + + hbase.hstore.compaction.throughput.lower.bound + 52428800 + The target lower bound on aggregate compaction throughput, in bytes/sec. Allows + you to tune the minimum available compaction throughput when the + PressureAwareCompactionThroughputController throughput controller is active. (It is active by + default.) + + + + hbase.hstore.compaction.throughput.higher.bound + 104857600 + The target upper bound on aggregate compaction throughput, in bytes/sec. Allows + you to control aggregate compaction throughput demand when the + PressureAwareCompactionThroughputController throughput controller is active. (It is active by + default.) The maximum throughput will be tuned between the lower and upper bounds when + compaction pressure is within the range [0.0, 1.0]. If compaction pressure is 1.0 or greater + the higher bound will be ignored until pressure returns to the normal range. + + + + hbase.bucketcache.size + + A float that EITHER represents a percentage of total heap memory + size to give to the cache (if < 1.0) OR, it is the total capacity in + megabytes of BucketCache. Default: 0.0 + + + + hbase.bucketcache.bucket.sizes + + A comma-separated list of sizes for buckets for the bucketcache. + Can be multiple sizes. List block sizes in order from smallest to largest. + The sizes you use will depend on your data access patterns. + Must be a multiple of 256 else you will run into + 'java.io.IOException: Invalid HFile block magic' when you go to read from cache. + If you specify no values here, then you pick up the default bucketsizes set + in code (See BucketAllocator#DEFAULT_BUCKET_SIZES). + + + + hfile.format.version + 3 + The HFile format version to use for new files. + Version 3 adds support for tags in hfiles (See http://hbase.apache.org/book.html#hbase.tags). + Also see the configuration 'hbase.replication.rpc.codec'. + + + + hfile.block.bloom.cacheonwrite + false + Enables cache-on-write for inline blocks of a compound Bloom filter. + + + io.storefile.bloom.block.size + 131072 + The size in bytes of a single block ("chunk") of a compound Bloom + filter. This size is approximate, because Bloom blocks can only be + inserted at data block boundaries, and the number of keys per data + block varies. + + + + hbase.rs.cacheblocksonwrite + false + Whether an HFile block should be added to the block cache when the + block is finished. + + + + hbase.rpc.timeout + 60000 + This is for the RPC layer to define how long (millisecond) HBase client applications + take for a remote call to time out. It uses pings to check connections + but will eventually throw a TimeoutException. + + + + hbase.client.operation.timeout + 1200000 + Operation timeout is a top-level restriction (millisecond) that makes sure a + blocking operation in Table will not be blocked more than this. In each operation, if rpc + request fails because of timeout or other reason, it will retry until success or throw + RetriesExhaustedException. But if the total time being blocking reach the operation timeout + before retries exhausted, it will break early and throw SocketTimeoutException. + + + + hbase.cells.scanned.per.heartbeat.check + 10000 + The number of cells scanned in between heartbeat checks. Heartbeat + checks occur during the processing of scans to determine whether or not the + server should stop scanning in order to send back a heartbeat message to the + client. Heartbeat messages are used to keep the client-server connection alive + during long running scans. Small values mean that the heartbeat checks will + occur more often and thus will provide a tighter bound on the execution time of + the scan. Larger values mean that the heartbeat checks occur less frequently + + + + hbase.rpc.shortoperation.timeout + 10000 + This is another version of "hbase.rpc.timeout". For those RPC operation + within cluster, we rely on this configuration to set a short timeout limitation + for short operation. For example, short rpc timeout for region server's trying + to report to active master can benefit quicker master failover process. + + + + hbase.ipc.client.tcpnodelay + true + Set no delay on rpc socket connections. See + http://docs.oracle.com/javase/1.5.0/docs/api/java/net/Socket.html#getTcpNoDelay() + + + + hbase.unsafe.regionserver.hostname + + This config is for experts: don't set its value unless you really know what you are doing. + When set to a non-empty value, this represents the (external facing) hostname for the underlying server. + See https://issues.apache.org/jira/browse/HBASE-12954 for details. + + + + hbase.unsafe.regionserver.hostname.disable.master.reversedns + false + This config is for experts: don't set its value unless you really know what you are doing. + When set to true, regionserver will use the current node hostname for the servername and HMaster will + skip reverse DNS lookup and use the hostname sent by regionserver instead. Note that this config and + hbase.unsafe.regionserver.hostname are mutually exclusive. See https://issues.apache.org/jira/browse/HBASE-18226 + for more details. + + + + + hbase.master.keytab.file + + Full path to the kerberos keytab file to use for logging in + the configured HMaster server principal. + + + + hbase.master.kerberos.principal + + Ex. "hbase/_HOST@EXAMPLE.COM". The kerberos principal name + that should be used to run the HMaster process. The principal name should + be in the form: user/hostname@DOMAIN. If "_HOST" is used as the hostname + portion, it will be replaced with the actual hostname of the running + instance. + + + + hbase.regionserver.keytab.file + + Full path to the kerberos keytab file to use for logging in + the configured HRegionServer server principal. + + + + hbase.regionserver.kerberos.principal + + Ex. "hbase/_HOST@EXAMPLE.COM". The kerberos principal name + that should be used to run the HRegionServer process. The principal name + should be in the form: user/hostname@DOMAIN. If "_HOST" is used as the + hostname portion, it will be replaced with the actual hostname of the + running instance. An entry for this principal must exist in the file + specified in hbase.regionserver.keytab.file + + + + + hadoop.policy.file + hbase-policy.xml + The policy configuration file used by RPC servers to make + authorization decisions on client requests. Only used when HBase + security is enabled. + + + + hbase.superuser + + List of users or groups (comma-separated), who are allowed + full privileges, regardless of stored ACLs, across the cluster. + Only used when HBase security is enabled. + + + + hbase.auth.key.update.interval + 86400000 + The update interval for master key for authentication tokens + in servers in milliseconds. Only used when HBase security is enabled. + + + + hbase.auth.token.max.lifetime + 604800000 + The maximum lifetime in milliseconds after which an + authentication token expires. Only used when HBase security is enabled. + + + + hbase.ipc.client.fallback-to-simple-auth-allowed + false + When a client is configured to attempt a secure connection, but attempts to + connect to an insecure server, that server may instruct the client to + switch to SASL SIMPLE (unsecure) authentication. This setting controls + whether or not the client will accept this instruction from the server. + When false (the default), the client will not allow the fallback to SIMPLE + authentication, and will abort the connection. + + + + hbase.ipc.server.fallback-to-simple-auth-allowed + false + When a server is configured to require secure connections, it will + reject connection attempts from clients using SASL SIMPLE (unsecure) authentication. + This setting allows secure servers to accept SASL SIMPLE connections from clients + when the client requests. When false (the default), the server will not allow the fallback + to SIMPLE authentication, and will reject the connection. WARNING: This setting should ONLY + be used as a temporary measure while converting clients over to secure authentication. It + MUST BE DISABLED for secure operation. + + + + hbase.display.keys + true + When this is set to true the webUI and such will display all start/end keys + as part of the table details, region names, etc. When this is set to false, + the keys are hidden. + + + + hbase.coprocessor.enabled + true + Enables or disables coprocessor loading. If 'false' + (disabled), any other coprocessor related configuration will be ignored. + + + + hbase.coprocessor.user.enabled + true + Enables or disables user (aka. table) coprocessor loading. + If 'false' (disabled), any table coprocessor attributes in table + descriptors will be ignored. If "hbase.coprocessor.enabled" is 'false' + this setting has no effect. + + + + hbase.coprocessor.region.classes + + A comma-separated list of Coprocessors that are loaded by + default on all tables. For any override coprocessor method, these classes + will be called in order. After implementing your own Coprocessor, just put + it in HBase's classpath and add the fully qualified class name here. + A coprocessor can also be loaded on demand by setting HTableDescriptor. + + + + hbase.coprocessor.master.classes + + A comma-separated list of + org.apache.hadoop.hbase.coprocessor.MasterObserver coprocessors that are + loaded by default on the active HMaster process. For any implemented + coprocessor methods, the listed classes will be called in order. After + implementing your own MasterObserver, just put it in HBase's classpath + and add the fully qualified class name here. + + + + hbase.coprocessor.abortonerror + true + Set to true to cause the hosting server (master or regionserver) + to abort if a coprocessor fails to load, fails to initialize, or throws an + unexpected Throwable object. Setting this to false will allow the server to + continue execution but the system wide state of the coprocessor in question + will become inconsistent as it will be properly executing in only a subset + of servers, so this is most useful for debugging only. + + + + hbase.rest.port + 8080 + The port for the HBase REST server. + + + hbase.rest.readonly + false + Defines the mode the REST server will be started in. Possible values are: + false: All HTTP methods are permitted - GET/PUT/POST/DELETE. + true: Only the GET method is permitted. + + + + hbase.rest.threads.max + 100 + The maximum number of threads of the REST server thread pool. + Threads in the pool are reused to process REST requests. This + controls the maximum number of requests processed concurrently. + It may help to control the memory used by the REST server to + avoid OOM issues. If the thread pool is full, incoming requests + will be queued up and wait for some free threads. + + + + hbase.rest.threads.min + 2 + The minimum number of threads of the REST server thread pool. + The thread pool always has at least these number of threads so + the REST server is ready to serve incoming requests. + + + + hbase.rest.support.proxyuser + false + Enables running the REST server to support proxy-user mode. + + + hbase.defaults.for.version + 2.4.9 + This defaults file was compiled for version ${project.version}. This variable is used + to make sure that a user doesn't have an old version of hbase-default.xml on the + classpath. + + + + hbase.defaults.for.version.skip + false + Set to true to skip the 'hbase.defaults.for.version' check. + Setting this to true can be useful in contexts other than + the other side of a maven generation; i.e. running in an + IDE. You'll want to set this boolean to true to avoid + seeing the RuntimeException complaint: "hbase-default.xml file + seems to be for and old version of HBase (\${hbase.version}), this + version is X.X.X-SNAPSHOT" + + + + hbase.table.lock.enable + true + Set to true to enable locking the table in zookeeper for schema change operations. + Table locking from master prevents concurrent schema modifications to corrupt table + state. + + + + hbase.table.max.rowsize + 1073741824 + + Maximum size of single row in bytes (default is 1 Gb) for Get'ting + or Scan'ning without in-row scan flag set. If row size exceeds this limit + RowTooBigException is thrown to client. + + + + hbase.thrift.minWorkerThreads + 16 + The "core size" of the thread pool. New threads are created on every + connection until this many threads are created. + + + + hbase.thrift.maxWorkerThreads + 1000 + The maximum size of the thread pool. When the pending request queue + overflows, new threads are created until their number reaches this number. + After that, the server starts dropping connections. + + + + hbase.thrift.maxQueuedRequests + 1000 + The maximum number of pending Thrift connections waiting in the queue. If + there are no idle threads in the pool, the server queues requests. Only + when the queue overflows, new threads are added, up to + hbase.thrift.maxQueuedRequests threads. + + + + hbase.regionserver.thrift.framed + false + Use Thrift TFramedTransport on the server side. + This is the recommended transport for thrift servers and requires a similar setting + on the client side. Changing this to false will select the default transport, + vulnerable to DoS when malformed requests are issued due to THRIFT-601. + + + + hbase.regionserver.thrift.framed.max_frame_size_in_mb + 2 + Default frame size when using framed transport, in MB + + + hbase.regionserver.thrift.compact + false + Use Thrift TCompactProtocol binary serialization protocol. + + + hbase.rootdir.perms + 700 + FS Permissions for the root data subdirectory in a secure (kerberos) setup. + When master starts, it creates the rootdir with this permissions or sets the permissions + if it does not match. + + + + hbase.wal.dir.perms + 700 + FS Permissions for the root WAL directory in a secure(kerberos) setup. + When master starts, it creates the WAL dir with this permissions or sets the permissions + if it does not match. + + + + hbase.data.umask.enable + false + Enable, if true, that file permissions should be assigned + to the files written by the regionserver + + + + hbase.data.umask + 000 + File permissions that should be used to write data + files when hbase.data.umask.enable is true + + + + hbase.snapshot.enabled + true + Set to true to allow snapshots to be taken / restored / cloned. + + + hbase.snapshot.restore.take.failsafe.snapshot + true + Set to true to take a snapshot before the restore operation. + The snapshot taken will be used in case of failure, to restore the previous state. + At the end of the restore operation this snapshot will be deleted + + + + hbase.snapshot.restore.failsafe.name + hbase-failsafe-{snapshot.name}-{restore.timestamp} + Name of the failsafe snapshot taken by the restore operation. + You can use the {snapshot.name}, {table.name} and {restore.timestamp} variables + to create a name based on what you are restoring. + + + + hbase.snapshot.working.dir + + Location where the snapshotting process will occur. The location of the + completed snapshots will not change, but the temporary directory where the snapshot + process occurs will be set to this location. This can be a separate filesystem than + the root directory, for performance increase purposes. See HBASE-21098 for more + information + + + + hbase.server.compactchecker.interval.multiplier + 1000 + The number that determines how often we scan to see if compaction is necessary. + Normally, compactions are done after some events (such as memstore flush), but if + region didn't receive a lot of writes for some time, or due to different compaction + policies, it may be necessary to check it periodically. The interval between checks is + hbase.server.compactchecker.interval.multiplier multiplied by + hbase.server.thread.wakefrequency. + + + + hbase.lease.recovery.timeout + 900000 + How long we wait on dfs lease recovery in total before giving up. + + + hbase.lease.recovery.dfs.timeout + 64000 + How long between dfs recover lease invocations. Should be larger than the sum of + the time it takes for the namenode to issue a block recovery command as part of + datanode; dfs.heartbeat.interval and the time it takes for the primary + datanode, performing block recovery to timeout on a dead datanode; usually + dfs.client.socket-timeout. See the end of HBASE-8389 for more. + + + + hbase.column.max.version + 1 + New column family descriptors will use this value as the default number of versions + to keep. + + + + dfs.client.read.shortcircuit + + + If set to true, this configuration parameter enables short-circuit local + reads. + + + + dfs.domain.socket.path + + + This is a path to a UNIX domain socket that will be used for + communication between the DataNode and local HDFS clients, if + dfs.client.read.shortcircuit is set to true. If the string "_PORT" is + present in this path, it will be replaced by the TCP port of the DataNode. + Be careful about permissions for the directory that hosts the shared + domain socket; dfsclient will complain if open to other users than the HBase user. + + + + hbase.dfs.client.read.shortcircuit.buffer.size + 131072 + If the DFSClient configuration + dfs.client.read.shortcircuit.buffer.size is unset, we will + use what is configured here as the short circuit read default + direct byte buffer size. DFSClient native default is 1MB; HBase + keeps its HDFS files open so number of file blocks * 1MB soon + starts to add up and threaten OOME because of a shortage of + direct memory. So, we set it down from the default. Make + it > the default hbase block size set in the HColumnDescriptor + which is usually 64k. + + + + hbase.regionserver.checksum.verify + true + + If set to true (the default), HBase verifies the checksums for hfile + blocks. HBase writes checksums inline with the data when it writes out + hfiles. HDFS (as of this writing) writes checksums to a separate file + than the data file necessitating extra seeks. Setting this flag saves + some on i/o. Checksum verification by HDFS will be internally disabled + on hfile streams when this flag is set. If the hbase-checksum verification + fails, we will switch back to using HDFS checksums (so do not disable HDFS + checksums! And besides this feature applies to hfiles only, not to WALs). + If this parameter is set to false, then hbase will not verify any checksums, + instead it will depend on checksum verification being done in the HDFS client. + + + + hbase.hstore.bytes.per.checksum + 16384 + + Number of bytes in a newly created checksum chunk for HBase-level + checksums in hfile blocks. + + + + hbase.hstore.checksum.algorithm + CRC32C + + Name of an algorithm that is used to compute checksums. Possible values + are NULL, CRC32, CRC32C. + + + + hbase.client.scanner.max.result.size + 2097152 + Maximum number of bytes returned when calling a scanner's next method. + Note that when a single row is larger than this limit the row is still returned completely. + The default value is 2MB, which is good for 1ge networks. + With faster and/or high latency networks this value should be increased. + + + + hbase.server.scanner.max.result.size + 104857600 + Maximum number of bytes returned when calling a scanner's next method. + Note that when a single row is larger than this limit the row is still returned completely. + The default value is 100MB. + This is a safety setting to protect the server from OOM situations. + + + + hbase.status.published + false + + This setting activates the publication by the master of the status of the region server. + When a region server dies and its recovery starts, the master will push this information + to the client application, to let them cut the connection immediately instead of waiting + for a timeout. + + + + hbase.status.publisher.class + org.apache.hadoop.hbase.master.ClusterStatusPublisher$MulticastPublisher + + Implementation of the status publication with a multicast message. + + + + hbase.status.listener.class + org.apache.hadoop.hbase.client.ClusterStatusListener$MulticastListener + + Implementation of the status listener with a multicast message. + + + + hbase.status.multicast.address.ip + 226.1.1.3 + + Multicast address to use for the status publication by multicast. + + + + hbase.status.multicast.address.port + 16100 + + Multicast port to use for the status publication by multicast. + + + + hbase.dynamic.jars.dir + ${hbase.rootdir}/lib + + The directory from which the custom filter JARs can be loaded + dynamically by the region server without the need to restart. However, + an already loaded filter/co-processor class would not be un-loaded. See + HBASE-1936 for more details. + + Does not apply to coprocessors. + + + + hbase.security.authentication + simple + + Controls whether or not secure authentication is enabled for HBase. + Possible values are 'simple' (no authentication), and 'kerberos'. + + + + hbase.rest.filter.classes + org.apache.hadoop.hbase.rest.filter.GzipFilter + + Servlet filters for REST service. + + + + hbase.master.loadbalancer.class + org.apache.hadoop.hbase.master.balancer.StochasticLoadBalancer + + Class used to execute the regions balancing when the period occurs. + See the class comment for more on how it works + http://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.html + It replaces the DefaultLoadBalancer as the default (since renamed + as the SimpleLoadBalancer). + + + + hbase.master.loadbalance.bytable + false + Factor Table name when the balancer runs. + Default: false. + + + + hbase.master.normalizer.class + org.apache.hadoop.hbase.master.normalizer.SimpleRegionNormalizer + + Class used to execute the region normalization when the period occurs. + See the class comment for more on how it works + http://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/master/normalizer/SimpleRegionNormalizer.html + + + + hbase.rest.csrf.enabled + false + + Set to true to enable protection against cross-site request forgery (CSRF) + + + + hbase.rest-csrf.browser-useragents-regex + ^Mozilla.*,^Opera.* + + A comma-separated list of regular expressions used to match against an HTTP + request's User-Agent header when protection against cross-site request + forgery (CSRF) is enabled for REST server by setting + hbase.rest.csrf.enabled to true. If the incoming User-Agent matches + any of these regular expressions, then the request is considered to be sent + by a browser, and therefore CSRF prevention is enforced. If the request's + User-Agent does not match any of these regular expressions, then the request + is considered to be sent by something other than a browser, such as scripted + automation. In this case, CSRF is not a potential attack vector, so + the prevention is not enforced. This helps achieve backwards-compatibility + with existing automation that has not been updated to send the CSRF + prevention header. + + + + hbase.security.exec.permission.checks + false + + If this setting is enabled and ACL based access control is active (the + AccessController coprocessor is installed either as a system coprocessor + or on a table as a table coprocessor) then you must grant all relevant + users EXEC privilege if they require the ability to execute coprocessor + endpoint calls. EXEC privilege, like any other permission, can be + granted globally to a user, or to a user on a per table or per namespace + basis. For more information on coprocessor endpoints, see the coprocessor + section of the HBase online manual. For more information on granting or + revoking permissions using the AccessController, see the security + section of the HBase online manual. + + + + hbase.procedure.regionserver.classes + + A comma-separated list of + org.apache.hadoop.hbase.procedure.RegionServerProcedureManager procedure managers that are + loaded by default on the active HRegionServer process. The lifecycle methods (init/start/stop) + will be called by the active HRegionServer process to perform the specific globally barriered + procedure. After implementing your own RegionServerProcedureManager, just put it in + HBase's classpath and add the fully qualified class name here. + + + + hbase.procedure.master.classes + + A comma-separated list of + org.apache.hadoop.hbase.procedure.MasterProcedureManager procedure managers that are + loaded by default on the active HMaster process. A procedure is identified by its signature and + users can use the signature and an instant name to trigger an execution of a globally barriered + procedure. After implementing your own MasterProcedureManager, just put it in HBase's classpath + and add the fully qualified class name here. + + + + hbase.coordinated.state.manager.class + org.apache.hadoop.hbase.coordination.ZkCoordinatedStateManager + Fully qualified name of class implementing coordinated state manager. + + + hbase.regionserver.storefile.refresh.period + 0 + + The period (in milliseconds) for refreshing the store files for the secondary regions. 0 + means this feature is disabled. Secondary regions sees new files (from flushes and + compactions) from primary once the secondary region refreshes the list of files in the + region (there is no notification mechanism). But too frequent refreshes might cause + extra Namenode pressure. If the files cannot be refreshed for longer than HFile TTL + (hbase.master.hfilecleaner.ttl) the requests are rejected. Configuring HFile TTL to a larger + value is also recommended with this setting. + + + + hbase.region.replica.replication.enabled + false + + Whether asynchronous WAL replication to the secondary region replicas is enabled or not. + If this is enabled, a replication peer named "region_replica_replication" will be created + which will tail the logs and replicate the mutations to region replicas for tables that + have region replication > 1. If this is enabled once, disabling this replication also + requires disabling the replication peer using shell or Admin java class. + Replication to secondary region replicas works over standard inter-cluster replication. + + + + hbase.http.filter.initializers + org.apache.hadoop.hbase.http.lib.StaticUserWebFilter + + A comma separated list of class names. Each class in the list must extend + org.apache.hadoop.hbase.http.FilterInitializer. The corresponding Filter will + be initialized. Then, the Filter will be applied to all user facing jsp + and servlet web pages. + The ordering of the list defines the ordering of the filters. + The default StaticUserWebFilter add a user principal as defined by the + hbase.http.staticuser.user property. + + + + hbase.security.visibility.mutations.checkauths + false + + This property if enabled, will check whether the labels in the visibility + expression are associated with the user issuing the mutation + + + + hbase.http.max.threads + 16 + + The maximum number of threads that the HTTP Server will create in its + ThreadPool. + + + + hbase.replication.rpc.codec + org.apache.hadoop.hbase.codec.KeyValueCodecWithTags + + The codec that is to be used when replication is enabled so that + the tags are also replicated. This is used along with HFileV3 which + supports tags in them. If tags are not used or if the hfile version used + is HFileV2 then KeyValueCodec can be used as the replication codec. Note that + using KeyValueCodecWithTags for replication when there are no tags causes no harm. + + + + hbase.replication.source.maxthreads + 10 + + The maximum number of threads any replication source will use for + shipping edits to the sinks in parallel. This also limits the number of + chunks each replication batch is broken into. Larger values can improve + the replication throughput between the master and slave clusters. The + default of 10 will rarely need to be changed. + + + + + hbase.http.staticuser.user + dr.stack + + The user name to filter as, on static web filters + while rendering content. An example use is the HDFS + web UI (user to be used for browsing files). + + + + hbase.regionserver.handler.abort.on.error.percent + 0.5 + The percent of region server RPC threads failed to abort RS. + -1 Disable aborting; 0 Abort if even a single handler has died; + 0.x Abort only when this percent of handlers have died; + 1 Abort only all of the handers have died. + + + + + hbase.mob.file.cache.size + 1000 + + Number of opened file handlers to cache. + A larger value will benefit reads by providing more file handlers per mob + file cache and would reduce frequent file opening and closing. + However, if this is set too high, this could lead to a "too many opened file handlers" + The default value is 1000. + + + + hbase.mob.cache.evict.period + 3600 + + The amount of time in seconds before the mob cache evicts cached mob files. + The default value is 3600 seconds. + + + + hbase.mob.cache.evict.remain.ratio + 0.5f + + The ratio (between 0.0 and 1.0) of files that remains cached after an eviction + is triggered when the number of cached mob files exceeds the hbase.mob.file.cache.size. + The default value is 0.5f. + + + + hbase.master.mob.ttl.cleaner.period + 86400 + + The period that ExpiredMobFileCleanerChore runs. The unit is second. + The default value is one day. The MOB file name uses only the date part of + the file creation time in it. We use this time for deciding TTL expiry of + the files. So the removal of TTL expired files might be delayed. The max + delay might be 24 hrs. + + + + hbase.mob.compaction.mergeable.threshold + 1342177280 + + If the size of a mob file is less than this value, it's regarded as a small + file and needs to be merged in mob compaction. The default value is 1280MB. + + + + hbase.mob.delfile.max.count + 3 + + The max number of del files that is allowed in the mob compaction. + In the mob compaction, when the number of existing del files is larger than + this value, they are merged until number of del files is not larger this value. + The default value is 3. + + + + hbase.mob.compaction.batch.size + 100 + + The max number of the mob files that is allowed in a batch of the mob compaction. + The mob compaction merges the small mob files to bigger ones. If the number of the + small files is very large, it could lead to a "too many opened file handlers" in the merge. + And the merge has to be split into batches. This value limits the number of mob files + that are selected in a batch of the mob compaction. The default value is 100. + + + + hbase.mob.compaction.chore.period + 604800 + + The period that MobCompactionChore runs. The unit is second. + The default value is one week. + + + + hbase.mob.compactor.class + org.apache.hadoop.hbase.mob.compactions.PartitionedMobCompactor + + Implementation of mob compactor, the default one is PartitionedMobCompactor. + + + + hbase.mob.compaction.threads.max + 1 + + The max number of threads used in MobCompactor. + + + + hbase.snapshot.master.timeout.millis + 300000 + + Timeout for master for the snapshot procedure execution. + + + + hbase.snapshot.region.timeout + 300000 + + Timeout for regionservers to keep threads in snapshot request pool waiting. + + + + hbase.rpc.rows.warning.threshold + 5000 + + Number of rows in a batch operation above which a warning will be logged. + + + + hbase.master.wait.on.service.seconds + 30 + Default is 5 minutes. Make it 30 seconds for tests. See + HBASE-19794 for some context. + + + + hbase.master.cleaner.snapshot.interval + 1800000 + + Snapshot Cleanup chore interval in milliseconds. + The cleanup thread keeps running at this interval + to find all snapshots that are expired based on TTL + and delete them. + + + + hbase.master.snapshot.ttl + 0 + + Default Snapshot TTL to be considered when the user does not specify TTL while + creating snapshot. Default value 0 indicates FOREVERE - snapshot should not be + automatically deleted until it is manually deleted + + + + hbase.master.regions.recovery.check.interval + 1200000 + + Regions Recovery Chore interval in milliseconds. + This chore keeps running at this interval to + find all regions with configurable max store file ref count + and reopens them. + + + + hbase.regions.recovery.store.file.ref.count + -1 + + Very large number of ref count on a compacted + store file indicates that it is a ref leak + on that object(compacted store file). + Such files can not be removed after + it is invalidated via compaction. + Only way to recover in such scenario is to + reopen the region which can release + all resources, like the refcount, + leases, etc. This config represents Store files Ref + Count threshold value considered for reopening + regions. Any region with compacted store files + ref count > this value would be eligible for + reopening by master. Here, we get the max + refCount among all refCounts on all + compacted away store files that belong to a + particular region. Default value -1 indicates + this feature is turned off. Only positive + integer value should be provided to + enable this feature. + + + + hbase.regionserver.slowlog.ringbuffer.size + 256 + + Default size of ringbuffer to be maintained by each RegionServer in order + to store online slowlog responses. This is an in-memory ring buffer of + requests that were judged to be too slow in addition to the responseTooSlow + logging. The in-memory representation would be complete. + For more details, please look into Doc Section: + Get Slow Response Log from shell + + + + hbase.regionserver.slowlog.buffer.enabled + false + + Indicates whether RegionServers have ring buffer running for storing + Online Slow logs in FIFO manner with limited entries. The size of + the ring buffer is indicated by config: hbase.regionserver.slowlog.ringbuffer.size + The default value is false, turn this on and get latest slowlog + responses with complete data. + + + + hbase.regionserver.slowlog.systable.enabled + false + + Should be enabled only if hbase.regionserver.slowlog.buffer.enabled is enabled. If enabled + (true), all slow/large RPC logs would be persisted to system table hbase:slowlog (in addition + to in-memory ring buffer at each RegionServer). The records are stored in increasing + order of time. Operators can scan the table with various combination of ColumnValueFilter. + More details are provided in the doc section: + "Get Slow/Large Response Logs from System table hbase:slowlog" + + + + hbase.rpc.rows.size.threshold.reject + false + + If value is true, RegionServer will abort batch requests of Put/Delete with number of rows + in a batch operation exceeding threshold defined by value of config: + hbase.rpc.rows.warning.threshold. The default value is false and hence, by default, only + warning will be logged. This config should be turned on to prevent RegionServer from serving + very large batch size of rows and this way we can improve CPU usages by discarding + too large batch request. + + + + hbase.namedqueue.provider.classes + + org.apache.hadoop.hbase.namequeues.impl.SlowLogQueueService,org.apache.hadoop.hbase.namequeues.impl.BalancerDecisionQueueService,org.apache.hadoop.hbase.namequeues.impl.BalancerRejectionQueueService + + + Default values for NamedQueueService implementors. This comma separated full class names + represent all implementors of NamedQueueService that we would like to be invoked by + LogEvent handler service. One example of NamedQueue service is SlowLogQueueService which + is used to store slow/large RPC logs in ringbuffer at each RegionServer. + All implementors of NamedQueueService should be found under package: + "org.apache.hadoop.hbase.namequeues.impl" + + + + hbase.master.balancer.decision.buffer.enabled + false + + Indicates whether active HMaster has ring buffer running for storing + balancer decisions in FIFO manner with limited entries. The size of + the ring buffer is indicated by config: hbase.master.balancer.decision.queue.size + + + + hbase.master.balancer.rejection.buffer.enabled + false + + Indicates whether active HMaster has ring buffer running for storing + balancer rejection in FIFO manner with limited entries. The size of + the ring buffer is indicated by config: hbase.master.balancer.rejection.queue.size + + + diff --git a/hudi-hadoop-mr/pom.xml b/hudi-hadoop-mr/pom.xml index 8aa378036d9c0..2b24f6a99ea95 100644 --- a/hudi-hadoop-mr/pom.xml +++ b/hudi-hadoop-mr/pom.xml @@ -128,14 +128,6 @@ - - - src/main/resources - - hbase-site.xml - - - org.apache.rat diff --git a/hudi-hadoop-mr/src/main/resources/hbase-site.xml b/hudi-hadoop-mr/src/main/resources/hbase-site.xml deleted file mode 100644 index 00b093727e3c5..0000000000000 --- a/hudi-hadoop-mr/src/main/resources/hbase-site.xml +++ /dev/null @@ -1,36 +0,0 @@ - - - - - - hbase.defaults.for.version.skip - true - Set to true to skip the 'hbase.defaults.for.version' check. - Setting this to true can be useful in contexts other than - the other side of a maven generation; i.e. running in an - IDE. You'll want to set this boolean to true to avoid - seeing the RuntimeException complaint: "hbase-default.xml file - seems to be for and old version of HBase (\${hbase.version}), this - version is X.X.X-SNAPSHOT" - - - \ No newline at end of file diff --git a/packaging/hudi-hadoop-mr-bundle/pom.xml b/packaging/hudi-hadoop-mr-bundle/pom.xml index 6f8a95a47e66a..5311e7479df0c 100644 --- a/packaging/hudi-hadoop-mr-bundle/pom.xml +++ b/packaging/hudi-hadoop-mr-bundle/pom.xml @@ -208,11 +208,6 @@ META-INF/*.SF META-INF/*.DSA META-INF/*.RSA - META-INF/maven/com.google.protobuf/** - META-INF/maven/commons-io/** - META-INF/maven/org.apache.hbase/** - META-INF/maven/org.apache.hbase.thirdparty/** - META-INF/maven/org.apache.htrace/** META-INF/services/javax.* **/*.proto hbase-webapps/** @@ -227,7 +222,7 @@ - src/main/resources + ../../hudi-common/src/main/resour src/test/resources From 6e0909f1c690e4ecb76ed76487350190cd099b14 Mon Sep 17 00:00:00 2001 From: Y Ethan Guo Date: Tue, 8 Mar 2022 22:03:43 -0800 Subject: [PATCH 20/36] Update presto setup in docker demo --- docker/hoodie/hadoop/sparkadhoc/Dockerfile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/hoodie/hadoop/sparkadhoc/Dockerfile b/docker/hoodie/hadoop/sparkadhoc/Dockerfile index 9e5a4cb68332b..c0e11012abadd 100644 --- a/docker/hoodie/hadoop/sparkadhoc/Dockerfile +++ b/docker/hoodie/hadoop/sparkadhoc/Dockerfile @@ -20,7 +20,7 @@ ARG HIVE_VERSION=2.3.3 ARG SPARK_VERSION=2.4.4 FROM apachehudi/hudi-hadoop_${HADOOP_VERSION}-hive_${HIVE_VERSION}-sparkbase_${SPARK_VERSION} -ARG PRESTO_VERSION=0.268 +ARG PRESTO_VERSION=0.271 ARG TRINO_VERSION=368 COPY adhoc.sh /opt/spark From 6f81ee726966cf89d7e5cee2413cea569388e2d6 Mon Sep 17 00:00:00 2001 From: Y Ethan Guo Date: Wed, 9 Mar 2022 09:46:31 -0800 Subject: [PATCH 21/36] Some cleanup in docker setup --- docker/hoodie/hadoop/hive_base/Dockerfile | 2 -- docker/hoodie/hadoop/sparkadhoc/Dockerfile | 2 +- 2 files changed, 1 insertion(+), 3 deletions(-) diff --git a/docker/hoodie/hadoop/hive_base/Dockerfile b/docker/hoodie/hadoop/hive_base/Dockerfile index cf40bdcadca8f..7d04d94fc60cc 100644 --- a/docker/hoodie/hadoop/hive_base/Dockerfile +++ b/docker/hoodie/hadoop/hive_base/Dockerfile @@ -36,8 +36,6 @@ RUN echo "Hive URL is :${HIVE_URL}" && wget ${HIVE_URL} -O hive.tar.gz && \ wget https://jdbc.postgresql.org/download/postgresql-9.4.1212.jar -O $HIVE_HOME/lib/postgresql-jdbc.jar && \ rm hive.tar.gz && mkdir -p /var/hoodie/ws/docker/hoodie/hadoop/hive_base/target/ -RUN rm hive/lib/hbase* - #Spark should be compiled with Hive to be able to use it #hive-site.xml should be copied to $SPARK_HOME/conf folder diff --git a/docker/hoodie/hadoop/sparkadhoc/Dockerfile b/docker/hoodie/hadoop/sparkadhoc/Dockerfile index c0e11012abadd..9e5a4cb68332b 100644 --- a/docker/hoodie/hadoop/sparkadhoc/Dockerfile +++ b/docker/hoodie/hadoop/sparkadhoc/Dockerfile @@ -20,7 +20,7 @@ ARG HIVE_VERSION=2.3.3 ARG SPARK_VERSION=2.4.4 FROM apachehudi/hudi-hadoop_${HADOOP_VERSION}-hive_${HIVE_VERSION}-sparkbase_${SPARK_VERSION} -ARG PRESTO_VERSION=0.271 +ARG PRESTO_VERSION=0.268 ARG TRINO_VERSION=368 COPY adhoc.sh /opt/spark From 2c8ad7858ad69c7c16f6914d82dee8342f7c75d5 Mon Sep 17 00:00:00 2001 From: Y Ethan Guo Date: Wed, 9 Mar 2022 19:29:35 -0800 Subject: [PATCH 22/36] Simplify dependency changes --- hudi-aws/pom.xml | 19 ------------------- hudi-client/hudi-java-client/pom.xml | 20 -------------------- hudi-client/hudi-spark-client/pom.xml | 6 ------ hudi-common/pom.xml | 2 +- hudi-hadoop-mr/pom.xml | 5 +++++ 5 files changed, 6 insertions(+), 46 deletions(-) diff --git a/hudi-aws/pom.xml b/hudi-aws/pom.xml index 4abbd119a0d55..d44a389a61f66 100644 --- a/hudi-aws/pom.xml +++ b/hudi-aws/pom.xml @@ -48,25 +48,6 @@ - - org.apache.hadoop - hadoop-common - provided - - - org.mortbay.jetty - * - - - javax.servlet.jsp - * - - - javax.servlet - * - - - org.apache.hadoop hadoop-common diff --git a/hudi-client/hudi-java-client/pom.xml b/hudi-client/hudi-java-client/pom.xml index b299150c6e3e0..3471bfb8ba366 100644 --- a/hudi-client/hudi-java-client/pom.xml +++ b/hudi-client/hudi-java-client/pom.xml @@ -122,26 +122,6 @@ test - - org.apache.hadoop - hadoop-hdfs - test - - - - org.mortbay.jetty - * - - - javax.servlet.jsp - * - - - javax.servlet - * - - - org.apache.hadoop hadoop-hdfs diff --git a/hudi-client/hudi-spark-client/pom.xml b/hudi-client/hudi-spark-client/pom.xml index 20cccda5ea420..0688fedacc2ae 100644 --- a/hudi-client/hudi-spark-client/pom.xml +++ b/hudi-client/hudi-spark-client/pom.xml @@ -90,12 +90,6 @@ - - org.apache.hadoop - hadoop-hdfs-client - ${hadoop.version} - test - org.apache.hbase hbase-testing-util diff --git a/hudi-common/pom.xml b/hudi-common/pom.xml index b5b2a3a47a4ae..a7de212e53a80 100644 --- a/hudi-common/pom.xml +++ b/hudi-common/pom.xml @@ -222,7 +222,7 @@ hbase-client ${hbase.version} - + org.apache.hbase hbase-server diff --git a/hudi-hadoop-mr/pom.xml b/hudi-hadoop-mr/pom.xml index 2b24f6a99ea95..bf87bfaa36a81 100644 --- a/hudi-hadoop-mr/pom.xml +++ b/hudi-hadoop-mr/pom.xml @@ -128,6 +128,11 @@ + + + src/main/resources + + org.apache.rat From 756d26a8055e09fc97db127db6a9d4afbfdb41f4 Mon Sep 17 00:00:00 2001 From: Y Ethan Guo Date: Thu, 10 Mar 2022 08:32:36 -0800 Subject: [PATCH 23/36] Trim bundle dependencies --- packaging/hudi-flink-bundle/pom.xml | 36 ++-------- packaging/hudi-hadoop-mr-bundle/pom.xml | 22 ++---- packaging/hudi-presto-bundle/pom.xml | 59 ++-------------- packaging/hudi-spark-bundle/pom.xml | 90 ++----------------------- packaging/hudi-trino-bundle/pom.xml | 60 ++--------------- packaging/hudi-utilities-bundle/pom.xml | 84 ++--------------------- 6 files changed, 26 insertions(+), 325 deletions(-) diff --git a/packaging/hudi-flink-bundle/pom.xml b/packaging/hudi-flink-bundle/pom.xml index f79e3f43fc053..39657d795ec4c 100644 --- a/packaging/hudi-flink-bundle/pom.xml +++ b/packaging/hudi-flink-bundle/pom.xml @@ -217,6 +217,10 @@ com.fasterxml.jackson. ${flink.bundle.shade.prefix}com.fasterxml.jackson. + org.apache.hadoop.metrics2.MetricHistogram org.apache.hudi.org.apache.hadoop.metrics2.MetricHistogram @@ -289,11 +293,6 @@ META-INF/*.SF META-INF/*.DSA META-INF/*.RSA - META-INF/maven/com.google.protobuf/** - META-INF/maven/commons-io/** - META-INF/maven/org.apache.hbase/** - META-INF/maven/org.apache.hbase.thirdparty/** - META-INF/maven/org.apache.htrace/** META-INF/services/javax.* **/*.proto hbase-webapps/** @@ -699,34 +698,9 @@ org.apache.hbase - hbase-protocol-shaded + hbase-protocol ${hbase.version} - - org.apache.hbase.thirdparty - hbase-shaded-miscellaneous - ${hbase-thirdparty.version} - - - org.apache.hbase.thirdparty - hbase-shaded-netty - ${hbase-thirdparty.version} - - - org.apache.hbase - hbase-hadoop-compat - ${hbase.version} - - - org.apache.hbase.thirdparty - hbase-shaded-protobuf - ${hbase-thirdparty.version} - - - org.apache.htrace - htrace-core - ${htrace.version} - diff --git a/packaging/hudi-hadoop-mr-bundle/pom.xml b/packaging/hudi-hadoop-mr-bundle/pom.xml index 5311e7479df0c..7115fbd0c4bef 100644 --- a/packaging/hudi-hadoop-mr-bundle/pom.xml +++ b/packaging/hudi-hadoop-mr-bundle/pom.xml @@ -135,6 +135,10 @@ com.google.common. org.apache.hudi.com.google.common. + org.apache.hadoop.metrics2.MetricHistogram org.apache.hudi.org.apache.hadoop.metrics2.MetricHistogram @@ -222,7 +226,7 @@ - ../../hudi-common/src/main/resour + src/main/resources src/test/resources @@ -258,21 +262,5 @@ ${avro.version} compile - - - org.apache.hbase - hbase-hadoop-compat - ${hbase.version} - - - org.apache.hbase - hbase-hadoop2-compat - ${hbase.version} - - - org.apache.hbase.thirdparty - hbase-shaded-protobuf - ${hbase-thirdparty.version} - diff --git a/packaging/hudi-presto-bundle/pom.xml b/packaging/hudi-presto-bundle/pom.xml index 51d24987df376..3c5e12e7f43cc 100644 --- a/packaging/hudi-presto-bundle/pom.xml +++ b/packaging/hudi-presto-bundle/pom.xml @@ -155,6 +155,10 @@ org.apache.parquet.avro. ${presto.bundle.bootstrap.shade.prefix}org.apache.parquet.avro. + org.apache.hadoop.metrics2.MetricHistogram org.apache.hudi.org.apache.hadoop.metrics2.MetricHistogram @@ -228,11 +232,6 @@ META-INF/*.SF META-INF/*.DSA META-INF/*.RSA - META-INF/maven/com.google.protobuf/** - META-INF/maven/commons-io/** - META-INF/maven/org.apache.hbase/** - META-INF/maven/org.apache.hbase.thirdparty/** - META-INF/maven/org.apache.htrace/** META-INF/services/javax.* **/*.proto com/esotericsoftware/reflectasm/** @@ -268,20 +267,6 @@ org.apache.hudi hudi-hadoop-mr-bundle ${project.version} - - - org.apache.hbase - hbase-common - - - org.apache.hbase - hbase-server - - - org.apache.hbase - hbase-client - - @@ -298,42 +283,6 @@ compile - - - org.apache.hbase - hbase-common - ${hbase.version} - - - - org.apache.hbase - hbase-server - ${hbase.version} - compile - - - org.apache.hbase - hbase-common - - - javax.servlet - * - - - org.codehaus.jackson - * - - - org.mortbay.jetty - * - - - tomcat - * - - - - diff --git a/packaging/hudi-spark-bundle/pom.xml b/packaging/hudi-spark-bundle/pom.xml index 54eb2a358f0e4..7e6f01ede94d0 100644 --- a/packaging/hudi-spark-bundle/pom.xml +++ b/packaging/hudi-spark-bundle/pom.xml @@ -208,6 +208,10 @@ ${spark.bundle.spark.shade.prefix}com.google.common. + org.apache.hadoop.metrics2.MetricHistogram org.apache.hudi.org.apache.hadoop.metrics2.MetricHistogram @@ -280,11 +284,6 @@ META-INF/*.SF META-INF/*.DSA META-INF/*.RSA - META-INF/maven/com.google.protobuf/** - META-INF/maven/commons-io/** - META-INF/maven/org.apache.hbase/** - META-INF/maven/org.apache.hbase.thirdparty/** - META-INF/maven/org.apache.htrace/** META-INF/services/javax.* **/*.proto hbase-webapps/** @@ -410,87 +409,6 @@ ${spark.bundle.hive.scope} - - org.apache.htrace - htrace-core - ${htrace.version} - compile - - - - - org.apache.hbase - hbase-common - ${hbase.version} - - - org.apache.hbase - hbase-server - ${hbase.version} - compile - - - guava - com.google.guava - - - org.apache.hbase - hbase-common - - - javax.servlet - * - - - org.codehaus.jackson - * - - - org.mortbay.jetty - * - - - tomcat - * - - - - - org.apache.hbase - hbase-client - ${hbase.version} - - - org.apache.hbase - hbase-protocol-shaded - ${hbase.version} - - - org.apache.hbase.thirdparty - hbase-shaded-miscellaneous - ${hbase-thirdparty.version} - - - org.apache.hbase.thirdparty - hbase-shaded-netty - ${hbase-thirdparty.version} - - - org.apache.hbase - hbase-hadoop-compat - ${hbase.version} - - - org.apache.hbase - hbase-hadoop2-compat - ${hbase.version} - - - org.apache.hbase.thirdparty - hbase-shaded-protobuf - ${hbase-thirdparty.version} - - org.apache.curator diff --git a/packaging/hudi-trino-bundle/pom.xml b/packaging/hudi-trino-bundle/pom.xml index 7deced3a323d9..68861cb6e8a0c 100644 --- a/packaging/hudi-trino-bundle/pom.xml +++ b/packaging/hudi-trino-bundle/pom.xml @@ -152,6 +152,10 @@ com.google.protobuf. ${trino.bundle.bootstrap.shade.prefix}com.google.protobuf. + org.apache.hadoop.metrics2.MetricHistogram org.apache.hudi.org.apache.hadoop.metrics2.MetricHistogram @@ -225,11 +229,6 @@ META-INF/*.SF META-INF/*.DSA META-INF/*.RSA - META-INF/maven/com.google.protobuf/** - META-INF/maven/commons-io/** - META-INF/maven/org.apache.hbase/** - META-INF/maven/org.apache.hbase.thirdparty/** - META-INF/maven/org.apache.htrace/** META-INF/services/javax.* **/*.proto hbase-webapps/** @@ -254,61 +253,10 @@ - - org.apache.hudi - hudi-common - ${project.version} - org.apache.hudi hudi-hadoop-mr-bundle ${project.version} - - - org.apache.hbase - hbase-server - - - org.apache.hbase - hbase-client - - - - - - - org.apache.hbase - hbase-common - ${hbase.version} - - - - org.apache.hbase - hbase-server - ${hbase.version} - compile - - - org.apache.hbase - hbase-common - - - javax.servlet - * - - - org.codehaus.jackson - * - - - org.mortbay.jetty - * - - - tomcat - * - - diff --git a/packaging/hudi-utilities-bundle/pom.xml b/packaging/hudi-utilities-bundle/pom.xml index 5432bc389fac0..2771992f3f6cf 100644 --- a/packaging/hudi-utilities-bundle/pom.xml +++ b/packaging/hudi-utilities-bundle/pom.xml @@ -232,6 +232,10 @@ org.eclipse.jetty. org.apache.hudi.org.eclipse.jetty. + org.apache.hadoop.metrics2.MetricHistogram org.apache.hudi.org.apache.hadoop.metrics2.MetricHistogram @@ -304,11 +308,6 @@ META-INF/*.SF META-INF/*.DSA META-INF/*.RSA - META-INF/maven/com.google.protobuf/** - META-INF/maven/commons-io/** - META-INF/maven/org.apache.hbase/** - META-INF/maven/org.apache.hbase.thirdparty/** - META-INF/maven/org.apache.htrace/** META-INF/services/javax.* **/*.proto hbase-webapps/** @@ -435,81 +434,6 @@ compile - - - org.apache.hbase - hbase-common - ${hbase.version} - - - guava - com.google.guava - - - - - org.apache.hbase - hbase-server - ${hbase.version} - compile - - - guava - com.google.guava - - - org.apache.hbase - hbase-common - - - javax.servlet - * - - - org.codehaus.jackson - * - - - org.mortbay.jetty - * - - - tomcat - * - - - - - org.apache.hbase - hbase-client - ${hbase.version} - - - org.apache.hbase - hbase-protocol-shaded - ${hbase.version} - - - org.apache.hbase.thirdparty - hbase-shaded-miscellaneous - ${hbase-thirdparty.version} - - - org.apache.hbase.thirdparty - hbase-shaded-netty - ${hbase-thirdparty.version} - - - org.apache.hbase - hbase-hadoop-compat - ${hbase.version} - - - org.apache.hbase.thirdparty - hbase-shaded-protobuf - ${hbase-thirdparty.version} - - org.apache.curator From 8980f8439fd3734a1d41dbf95464bbc839cb7d89 Mon Sep 17 00:00:00 2001 From: Y Ethan Guo Date: Thu, 10 Mar 2022 14:43:35 -0800 Subject: [PATCH 24/36] Make adjustment to bundles --- packaging/hudi-flink-bundle/pom.xml | 71 ++++++------------------ packaging/hudi-integ-test-bundle/pom.xml | 3 - packaging/hudi-presto-bundle/pom.xml | 1 - 3 files changed, 16 insertions(+), 59 deletions(-) diff --git a/packaging/hudi-flink-bundle/pom.xml b/packaging/hudi-flink-bundle/pom.xml index 39657d795ec4c..68767b656b8d4 100644 --- a/packaging/hudi-flink-bundle/pom.xml +++ b/packaging/hudi-flink-bundle/pom.xml @@ -368,11 +368,23 @@ org.apache.hudi hudi-hadoop-mr ${project.version} + + + guava + com.google.guava + + org.apache.hudi hudi-hive-sync ${project.version} + + + guava + com.google.guava + + org.apache.hudi @@ -383,6 +395,10 @@ rocksdbjni org.rocksdb + + guava + com.google.guava + @@ -646,61 +662,6 @@ jackson-annotations compile - - - - org.apache.hbase - hbase-common - ${hbase.version} - - - guava - com.google.guava - - - - - org.apache.hbase - hbase-server - ${hbase.version} - compile - - - guava - com.google.guava - - - org.apache.hbase - hbase-common - - - javax.servlet - * - - - org.codehaus.jackson - * - - - org.mortbay.jetty - * - - - tomcat - * - - - - - org.apache.hbase - hbase-client - ${hbase.version} - - - org.apache.hbase - hbase-protocol - ${hbase.version} - diff --git a/packaging/hudi-integ-test-bundle/pom.xml b/packaging/hudi-integ-test-bundle/pom.xml index e11212a4a34a7..6b0318f029eb8 100644 --- a/packaging/hudi-integ-test-bundle/pom.xml +++ b/packaging/hudi-integ-test-bundle/pom.xml @@ -62,9 +62,6 @@ META-INF/services/org.apache.spark.sql.sources.DataSourceRegister - - hbase-default.xml - diff --git a/packaging/hudi-presto-bundle/pom.xml b/packaging/hudi-presto-bundle/pom.xml index 3c5e12e7f43cc..b50c79fe33063 100644 --- a/packaging/hudi-presto-bundle/pom.xml +++ b/packaging/hudi-presto-bundle/pom.xml @@ -82,7 +82,6 @@ org.apache.hbase:hbase-metrics-api org.apache.hbase:hbase-protocol org.apache.hbase:hbase-protocol-shaded - org.apache.hbase:hbase-annotations org.apache.hbase.thirdparty:hbase-shaded-miscellaneous org.apache.hbase.thirdparty:hbase-shaded-netty org.apache.hbase.thirdparty:hbase-shaded-protobuf From 4ecc97ff9f20844b710ba85e12896ef2ee64e8e9 Mon Sep 17 00:00:00 2001 From: Y Ethan Guo Date: Thu, 10 Mar 2022 14:56:34 -0800 Subject: [PATCH 25/36] Change kafka-connect-bundle and timeline-server-bundle --- packaging/hudi-kafka-connect-bundle/pom.xml | 124 +++++++++++++++--- packaging/hudi-timeline-server-bundle/pom.xml | 100 +++++++++++++- 2 files changed, 207 insertions(+), 17 deletions(-) diff --git a/packaging/hudi-kafka-connect-bundle/pom.xml b/packaging/hudi-kafka-connect-bundle/pom.xml index f66bc7f051e48..9e131cc76e517 100644 --- a/packaging/hudi-kafka-connect-bundle/pom.xml +++ b/packaging/hudi-kafka-connect-bundle/pom.xml @@ -58,14 +58,16 @@ implementation="org.apache.maven.plugins.shade.resource.ApacheLicenseResourceTransformer"> + implementation="org.apache.maven.plugins.shade.resource.ApacheNoticeResourceTransformer"> true + implementation="org.apache.maven.plugins.shade.resource.IncludeResourceTransformer"> META-INF/LICENSE target/classes/META-INF/LICENSE + @@ -115,13 +117,21 @@ org.objenesis:objenesis com.esotericsoftware:kryo-shaded com.esotericsoftware:minlog - + org.apache.hbase:hbase-client org.apache.hbase:hbase-common - org.apache.hbase:hbase-protocol + org.apache.hbase:hbase-hadoop-compat + org.apache.hbase:hbase-hadoop2-compat + org.apache.hbase:hbase-metrics + org.apache.hbase:hbase-metrics-api + org.apache.hbase:hbase-protocol-shaded org.apache.hbase:hbase-server - org.apache.htrace:htrace-core + org.apache.hbase.thirdparty:hbase-shaded-miscellaneous + org.apache.hbase.thirdparty:hbase-shaded-netty + org.apache.hbase.thirdparty:hbase-shaded-protobuf + org.apache.htrace:htrace-core4 org.scala-lang:* + commons-io:commons-io @@ -131,15 +141,104 @@ com.yammer.metrics. - ${kafka.connect.bundle.shade.prefix}com.yammer.metrics. + ${kafka.connect.bundle.shade.prefix}com.yammer.metrics. + com.beust.jcommander. - ${kafka.connect.bundle.shade.prefix}com.beust.jcommander. + ${kafka.connect.bundle.shade.prefix}com.beust.jcommander. + org.eclipse.jetty. - ${kafka.connect.bundle.shade.prefix}org.eclipse.jetty. + ${kafka.connect.bundle.shade.prefix}org.eclipse.jetty. + + + + org.apache.commons.io. + org.apache.hudi.org.apache.commons.io. + + + org.apache.hadoop.hbase. + org.apache.hudi.org.apache.hadoop.hbase. + + + org.apache.hbase. + org.apache.hudi.org.apache.hbase. + + + org.apache.htrace. + org.apache.hudi.org.apache.htrace. + + + + org.apache.hadoop.metrics2.MetricHistogram + org.apache.hudi.org.apache.hadoop.metrics2.MetricHistogram + + + + org.apache.hadoop.metrics2.MetricsExecutor + org.apache.hudi.org.apache.hadoop.metrics2.MetricsExecutor + + + + org.apache.hadoop.metrics2.impl.JmxCacheBuster + org.apache.hudi.org.apache.hadoop.metrics2.impl.JmxCacheBuster + + + + org.apache.hadoop.metrics2.lib.DefaultMetricsSystemHelper + + org.apache.hudi.org.apache.hadoop.metrics2.lib.DefaultMetricsSystemHelper + + + + org.apache.hadoop.metrics2.lib.DynamicMetricsRegistry + org.apache.hudi.org.apache.hadoop.metrics2.lib.DynamicMetricsRegistry + + + + org.apache.hadoop.metrics2.lib.MetricsExecutorImpl + org.apache.hudi.org.apache.hadoop.metrics2.lib.MetricsExecutorImpl + + + + org.apache.hadoop.metrics2.lib.MutableFastCounter + org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableFastCounter + + + + org.apache.hadoop.metrics2.lib.MutableHistogram + org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableHistogram + + + + org.apache.hadoop.metrics2.lib.MutableRangeHistogram + org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableRangeHistogram + + + + org.apache.hadoop.metrics2.lib.MutableSizeHistogram + org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableSizeHistogram + + + + org.apache.hadoop.metrics2.lib.MutableTimeHistogram + org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableTimeHistogram + + + + org.apache.hadoop.metrics2.util.MetricQuantile + org.apache.hudi.org.apache.hadoop.metrics2.util.MetricQuantile + + + + org.apache.hadoop.metrics2.util.MetricSampleQuantiles + org.apache.hudi.org.apache.hadoop.metrics2.util.MetricSampleQuantiles + @@ -150,6 +249,8 @@ META-INF/*.DSA META-INF/*.RSA META-INF/services/javax.* + **/*.proto + hbase-webapps/** @@ -322,13 +423,6 @@ ${utilities.bundle.hive.scope} - - org.apache.htrace - htrace-core - ${htrace.version} - compile - - diff --git a/packaging/hudi-timeline-server-bundle/pom.xml b/packaging/hudi-timeline-server-bundle/pom.xml index 18f7c9665a20f..99664399c3407 100644 --- a/packaging/hudi-timeline-server-bundle/pom.xml +++ b/packaging/hudi-timeline-server-bundle/pom.xml @@ -155,6 +155,8 @@ META-INF/*.DSA META-INF/*.RSA META-INF/services/javax.* + **/*.proto + hbase-webapps/** @@ -198,17 +200,111 @@ com.fasterxml.jackson.core:jackson-annotations com.fasterxml.jackson.core:jackson-core com.fasterxml.jackson.core:jackson-databind - org.apache.htrace:htrace-core org.apache.hbase:hbase-common org.apache.hbase:hbase-client - org.apache.hbase:hbase-protocol + org.apache.hbase:hbase-hadoop-compat + org.apache.hbase:hbase-hadoop2-compat + org.apache.hbase:hbase-metrics + org.apache.hbase:hbase-metrics-api + org.apache.hbase:hbase-protocol-shaded org.apache.hbase:hbase-server + org.apache.hbase.thirdparty:hbase-shaded-miscellaneous + org.apache.hbase.thirdparty:hbase-shaded-netty + org.apache.hbase.thirdparty:hbase-shaded-protobuf + org.apache.htrace:htrace-core4 com.esotericsoftware:kryo-shaded com.esotericsoftware:minlog + commons-io:commons-io log4j:log4j org.objenesis:objenesis + + + org.apache.commons.io. + org.apache.hudi.org.apache.commons.io. + + + org.apache.hadoop.hbase. + org.apache.hudi.org.apache.hadoop.hbase. + + + org.apache.hbase. + org.apache.hudi.org.apache.hbase. + + + org.apache.htrace. + org.apache.hudi.org.apache.htrace. + + + + org.apache.hadoop.metrics2.MetricHistogram + org.apache.hudi.org.apache.hadoop.metrics2.MetricHistogram + + + + org.apache.hadoop.metrics2.MetricsExecutor + org.apache.hudi.org.apache.hadoop.metrics2.MetricsExecutor + + + + org.apache.hadoop.metrics2.impl.JmxCacheBuster + org.apache.hudi.org.apache.hadoop.metrics2.impl.JmxCacheBuster + + + org.apache.hadoop.metrics2.lib.DefaultMetricsSystemHelper + org.apache.hudi.org.apache.hadoop.metrics2.lib.DefaultMetricsSystemHelper + + + + org.apache.hadoop.metrics2.lib.DynamicMetricsRegistry + org.apache.hudi.org.apache.hadoop.metrics2.lib.DynamicMetricsRegistry + + + + org.apache.hadoop.metrics2.lib.MetricsExecutorImpl + org.apache.hudi.org.apache.hadoop.metrics2.lib.MetricsExecutorImpl + + + + org.apache.hadoop.metrics2.lib.MutableFastCounter + org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableFastCounter + + + + org.apache.hadoop.metrics2.lib.MutableHistogram + org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableHistogram + + + + org.apache.hadoop.metrics2.lib.MutableRangeHistogram + org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableRangeHistogram + + + + org.apache.hadoop.metrics2.lib.MutableSizeHistogram + org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableSizeHistogram + + + + org.apache.hadoop.metrics2.lib.MutableTimeHistogram + org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableTimeHistogram + + + + org.apache.hadoop.metrics2.util.MetricQuantile + org.apache.hudi.org.apache.hadoop.metrics2.util.MetricQuantile + + + + org.apache.hadoop.metrics2.util.MetricSampleQuantiles + org.apache.hudi.org.apache.hadoop.metrics2.util.MetricSampleQuantiles + + + From ea9eabe45cf845a8c7a6fa85f23c53eb05b553ea Mon Sep 17 00:00:00 2001 From: Y Ethan Guo Date: Fri, 11 Mar 2022 17:23:35 -0800 Subject: [PATCH 26/36] Adjust hudi-integ-test-bundle --- packaging/hudi-integ-test-bundle/pom.xml | 102 ++++++++++++++++++++++- 1 file changed, 101 insertions(+), 1 deletion(-) diff --git a/packaging/hudi-integ-test-bundle/pom.xml b/packaging/hudi-integ-test-bundle/pom.xml index 6b0318f029eb8..fdab05db1164d 100644 --- a/packaging/hudi-integ-test-bundle/pom.xml +++ b/packaging/hudi-integ-test-bundle/pom.xml @@ -62,6 +62,7 @@ META-INF/services/org.apache.spark.sql.sources.DataSourceRegister + @@ -85,6 +86,20 @@ org.apache.hudi:hudi-aws org.apache.hudi:hudi-integ-test + org.apache.hbase:hbase-common + org.apache.hbase:hbase-client + org.apache.hbase:hbase-hadoop-compat + org.apache.hbase:hbase-hadoop2-compat + org.apache.hbase:hbase-metrics + org.apache.hbase:hbase-metrics-api + org.apache.hbase:hbase-protocol-shaded + org.apache.hbase:hbase-server + org.apache.hbase.thirdparty:hbase-shaded-miscellaneous + org.apache.hbase.thirdparty:hbase-shaded-netty + org.apache.hbase.thirdparty:hbase-shaded-protobuf + org.apache.htrace:htrace-core4 + commons-io:commons-io + org.jetbrains.kotlin:kotlin-stdlib-jdk8 org.jetbrains.kotlin:kotlin-stdlib org.jetbrains.kotlin:kotlin-stdlib-common @@ -155,7 +170,6 @@ com.fasterxml.jackson.core:jackson-databind com.fasterxml.jackson.dataformat:jackson-dataformat-yaml - org.apache.htrace:htrace-core org.apache.curator:curator-framework org.apache.curator:curator-client org.apache.curator:curator-recipes @@ -178,6 +192,22 @@ org.apache.commons.pool. org.apache.hudi.org.apache.commons.pool. + + org.apache.commons.io. + org.apache.hudi.org.apache.commons.io. + + + org.apache.hadoop.hbase. + org.apache.hudi.org.apache.hadoop.hbase. + + + org.apache.hbase. + org.apache.hudi.org.apache.hbase. + + + org.apache.htrace. + org.apache.hudi.org.apache.htrace. + org.apache.hive.jdbc. org.apache.hudi.org.apache.hive.jdbc. @@ -258,6 +288,74 @@ org.apache.parquet.avro. org.apache.hudi.org.apache.parquet.avro. + + + org.apache.hadoop.metrics2.MetricHistogram + org.apache.hudi.org.apache.hadoop.metrics2.MetricHistogram + + + + org.apache.hadoop.metrics2.MetricsExecutor + org.apache.hudi.org.apache.hadoop.metrics2.MetricsExecutor + + + + org.apache.hadoop.metrics2.impl.JmxCacheBuster + org.apache.hudi.org.apache.hadoop.metrics2.impl.JmxCacheBuster + + + org.apache.hadoop.metrics2.lib.DefaultMetricsSystemHelper + org.apache.hudi.org.apache.hadoop.metrics2.lib.DefaultMetricsSystemHelper + + + + org.apache.hadoop.metrics2.lib.DynamicMetricsRegistry + org.apache.hudi.org.apache.hadoop.metrics2.lib.DynamicMetricsRegistry + + + + org.apache.hadoop.metrics2.lib.MetricsExecutorImpl + org.apache.hudi.org.apache.hadoop.metrics2.lib.MetricsExecutorImpl + + + + org.apache.hadoop.metrics2.lib.MutableFastCounter + org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableFastCounter + + + + org.apache.hadoop.metrics2.lib.MutableHistogram + org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableHistogram + + + + org.apache.hadoop.metrics2.lib.MutableRangeHistogram + org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableRangeHistogram + + + + org.apache.hadoop.metrics2.lib.MutableSizeHistogram + org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableSizeHistogram + + + + org.apache.hadoop.metrics2.lib.MutableTimeHistogram + org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableTimeHistogram + + + + org.apache.hadoop.metrics2.util.MetricQuantile + org.apache.hudi.org.apache.hadoop.metrics2.util.MetricQuantile + + + + org.apache.hadoop.metrics2.util.MetricSampleQuantiles + org.apache.hudi.org.apache.hadoop.metrics2.util.MetricSampleQuantiles + + @@ -269,6 +367,8 @@ META-INF/NOTICE* META-INF/LICENSE* + **/*.proto + hbase-webapps/** From 2cf29a5a4a285b18d062e7f41c8c31157aa44bbd Mon Sep 17 00:00:00 2001 From: Y Ethan Guo Date: Thu, 17 Mar 2022 18:25:36 -0700 Subject: [PATCH 27/36] Improve HFile code logic --- .../apache/hudi/io/HoodieAppendHandle.java | 3 +- .../bootstrap/index/HFileBootstrapIndex.java | 3 +- .../common/table/log/HoodieLogFileReader.java | 4 +- .../table/log/block/HoodieHFileDataBlock.java | 15 +++- .../hudi/io/storage/HoodieHFileReader.java | 46 +++------- .../hudi/io/storage/HoodieHFileUtils.java | 87 +++++++++++++++++++ .../TestInLineFileSystemHFileInLining.java | 8 +- .../functional/TestHoodieLogFormat.java | 7 +- .../hadoop/testutils/InputFormatTestUtil.java | 9 +- 9 files changed, 133 insertions(+), 49 deletions(-) create mode 100644 hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileUtils.java diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieAppendHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieAppendHandle.java index a58e4d65d2879..f8081019683e7 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieAppendHandle.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieAppendHandle.java @@ -548,7 +548,8 @@ private static HoodieLogBlock getBlock(HoodieWriteConfig writeConfig, 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: diff --git a/hudi-common/src/main/java/org/apache/hudi/common/bootstrap/index/HFileBootstrapIndex.java b/hudi-common/src/main/java/org/apache/hudi/common/bootstrap/index/HFileBootstrapIndex.java index 7f36a47a4d24c..b8a2c202cb6c9 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/bootstrap/index/HFileBootstrapIndex.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/bootstrap/index/HFileBootstrapIndex.java @@ -33,6 +33,7 @@ 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; @@ -179,7 +180,7 @@ private static String getUserKeyFromCellKey(String cellKey) { private static HFile.Reader createReader(String hFilePath, Configuration conf, FileSystem fileSystem) { try { LOG.info("Opening HFile for reading :" + hFilePath); - return HFile.createReader(fileSystem, new HFilePathForReader(hFilePath), new CacheConfig(conf), true, conf); + return HoodieHFileUtils.createHFileReader(fileSystem, new HFilePathForReader(hFilePath), new CacheConfig(conf), conf); } catch (IOException ioe) { throw new HoodieIOException(ioe.getMessage(), ioe); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieLogFileReader.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieLogFileReader.java index 07cb36bb169bb..9555f9f08dead 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieLogFileReader.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieLogFileReader.java @@ -79,7 +79,7 @@ public class HoodieLogFileReader implements HoodieLogFormat.Reader { private long reverseLogFilePosition; private long lastReverseLogFilePosition; private boolean reverseReader; - private boolean enableRecordLookups; + private final boolean enableRecordLookups; private boolean closed = false; private transient Thread shutdownThread = null; @@ -208,7 +208,7 @@ private HoodieLogBlock readBlock() throws IOException { 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, diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieHFileDataBlock.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieHFileDataBlock.java index e843ad74cb31c..cdff7aeaf9377 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieHFileDataBlock.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieHFileDataBlock.java @@ -19,6 +19,7 @@ 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; @@ -65,6 +66,9 @@ public class HoodieHFileDataBlock extends HoodieDataBlock { private static final int DEFAULT_BLOCK_SIZE = 1024 * 1024; private final Option 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 content, @@ -73,16 +77,20 @@ public HoodieHFileDataBlock(FSDataInputStream inputStream, Option readerSchema, Map header, Map 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 records, Map 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 @@ -156,7 +164,8 @@ protected ClosableIterator deserializeRecords(byte[] content) thr Schema writerSchema = new Schema.Parser().parse(super.getLogBlockHeader().get(HeaderMetadataType.SCHEMA)); // Read the content - HoodieHFileReader reader = new HoodieHFileReader<>(content); + HoodieHFileReader reader = new HoodieHFileReader<>( + FSUtils.getFs(pathForReader.toString(), new Configuration()), pathForReader, content); // Sets up the writer schema reader.withSchema(writerSchema); Iterator recordIterator = reader.getRecordIterator(readerSchema); diff --git a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileReader.java b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileReader.java index 5c861c9cc7a26..05c78dab7aa6d 100644 --- a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileReader.java +++ b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileReader.java @@ -39,14 +39,10 @@ import org.apache.hadoop.fs.Seekable; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.hbase.fs.HFileSystem; -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.io.hfile.ReaderContext; -import org.apache.hadoop.hbase.io.hfile.ReaderContextBuilder; import org.apache.hadoop.hbase.nio.ByteBuff; import org.apache.hadoop.hbase.util.Pair; import org.apache.log4j.LogManager; @@ -65,7 +61,15 @@ import java.util.stream.Collectors; public class HoodieHFileReader implements HoodieFileReader { + 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; @@ -75,51 +79,27 @@ public class HoodieHFileReader 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, true, 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, true, 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); - FSDataInputStreamWrapper stream = new FSDataInputStreamWrapper(fsdis); - FileSystem fs = FSUtils.getFs("hoodie", conf); - HFileSystem hfs = (fs instanceof HFileSystem) ? (HFileSystem) fs : new HFileSystem(fs); - ReaderContext context = new ReaderContextBuilder() - .withFilePath(path) - .withInputStreamWrapper(stream) - .withFileSize(content.length) - .withFileSystem(hfs) - .withPrimaryReplicaReader(true) - .withReaderType(ReaderContext.ReaderType.STREAM) - .build(); - HFileInfo fileInfo = new HFileInfo(context, conf); - this.reader = HFile.createReader(context, fileInfo, new CacheConfig(conf), conf); - fileInfo.initMetaAndIndex(reader); + public HoodieHFileReader(FileSystem fs, Path path, byte[] content) throws IOException { + this.reader = HoodieHFileUtils.createHFileReader(fs, path, content); } @Override public String[] readMinMaxRecordKeys() { HFileInfo fileInfo = reader.getHFileInfo(); - return new String[] { new String(fileInfo.get(KEY_MIN_RECORD.getBytes())), + return new String[] {new String(fileInfo.get(KEY_MIN_RECORD.getBytes())), new String(fileInfo.get(KEY_MAX_RECORD.getBytes()))}; } diff --git a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileUtils.java b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileUtils.java new file mode 100644 index 0000000000000..ebacf2977e752 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileUtils.java @@ -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 path 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 path, 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(path) + .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; + } +} diff --git a/hudi-common/src/test/java/org/apache/hudi/common/fs/inline/TestInLineFileSystemHFileInLining.java b/hudi-common/src/test/java/org/apache/hudi/common/fs/inline/TestInLineFileSystemHFileInLining.java index f09ecf76b2d88..1abbf01c1163f 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/fs/inline/TestInLineFileSystemHFileInLining.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/fs/inline/TestInLineFileSystemHFileInLining.java @@ -60,7 +60,7 @@ public class TestInLineFileSystemHFileInLining { private static final String LOCAL_FORMATTER = "%010d"; private static final String VALUE_PREFIX = "value"; - private static final int MIN_BLOCK_SIZE = 1024; + private static final int MIN_BLOCK_BYTES = 1024; private final Configuration inMemoryConf; private final Configuration inlineConf; private final int maxRows = 100 + RANDOM.nextInt(1000); @@ -91,7 +91,7 @@ public void testSimpleInlineFileSystem() throws IOException { CacheConfig cacheConf = new CacheConfig(inMemoryConf); FSDataOutputStream fout = createFSOutput(outerInMemFSPath, inMemoryConf); HFileContext meta = new HFileContextBuilder() - .withBlockSize(MIN_BLOCK_SIZE).withCellComparator(COMPARATOR) + .withBlockSize(MIN_BLOCK_BYTES).withCellComparator(COMPARATOR) .build(); HFile.Writer writer = HFile.getWriterFactory(inMemoryConf, cacheConf) .withOutputStream(fout) @@ -129,8 +129,8 @@ public void testSimpleInlineFileSystem() throws IOException { // read the key and see if it matches Cell cell = scanner.getCell(); byte[] key = Arrays.copyOfRange(cell.getRowArray(), cell.getRowOffset(), cell.getRowOffset() + cell.getRowLength()); - assertArrayEquals(Arrays.copyOfRange(keyValue.getRowArray(), keyValue.getRowOffset(), keyValue.getRowOffset() + keyValue.getRowLength()), key, - "seeked key does not match"); + 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); diff --git a/hudi-common/src/test/java/org/apache/hudi/common/functional/TestHoodieLogFormat.java b/hudi-common/src/test/java/org/apache/hudi/common/functional/TestHoodieLogFormat.java index e9b06e6d6397d..0772dc63b1a89 100755 --- a/hudi-common/src/test/java/org/apache/hudi/common/functional/TestHoodieLogFormat.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/functional/TestHoodieLogFormat.java @@ -1886,11 +1886,16 @@ public void testDataBlockFormatAppendAndReadWithProjectedSchema( private HoodieDataBlock getDataBlock(HoodieLogBlockType dataBlockType, List records, Map header) { + return getDataBlock(dataBlockType, records, header, new Path("dummy_path")); + } + + private HoodieDataBlock getDataBlock(HoodieLogBlockType dataBlockType, List records, + Map 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: diff --git a/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/testutils/InputFormatTestUtil.java b/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/testutils/InputFormatTestUtil.java index 1185be65c196e..755793871165e 100644 --- a/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/testutils/InputFormatTestUtil.java +++ b/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/testutils/InputFormatTestUtil.java @@ -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.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 static HoodieLogFormat.Writer writeDataBlockToLogFile(File partitionDir, 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 { From 7de5986acb001a01bc2a28b4143431d4d568445e Mon Sep 17 00:00:00 2001 From: Y Ethan Guo Date: Fri, 18 Mar 2022 15:16:11 -0700 Subject: [PATCH 28/36] Restructure tests for HoodieFileReader and FoodieFileWriter and add more unit tests in TestHoodieHFileReaderWriter --- .../io/storage/HoodieFileWriterFactory.java | 26 +- .../storage/TestHoodieHFileReaderWriter.java | 129 ++++++--- .../io/storage/TestHoodieOrcReaderWriter.java | 233 +++------------- .../storage/TestHoodieReaderWriterBase.java | 250 ++++++++++++++++++ 4 files changed, 388 insertions(+), 250 deletions(-) create mode 100644 hudi-client/hudi-client-common/src/test/java/org/apache/hudi/io/storage/TestHoodieReaderWriterBase.java diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieFileWriterFactory.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieFileWriterFactory.java index 38db1cde41226..7d0c307dbfe53 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieFileWriterFactory.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieFileWriterFactory.java @@ -30,6 +30,7 @@ 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 static 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 static private static HoodieFileWriter 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 HoodieFileWriter 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 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 HoodieFileWriter newHFileFileWriter( - String instantTime, Path path, HoodieWriteConfig config, Schema schema, HoodieTable hoodieTable, + static HoodieFileWriter 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 @@ private static HoodieFi } private static HoodieFileWriter 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); } diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/io/storage/TestHoodieHFileReaderWriter.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/io/storage/TestHoodieHFileReaderWriter.java index fd25d92cba62e..672a5f9b7f9dc 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/io/storage/TestHoodieHFileReaderWriter.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/io/storage/TestHoodieHFileReaderWriter.java @@ -18,39 +18,39 @@ 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.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.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.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; @@ -59,30 +59,61 @@ 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 { - @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 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 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 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 populateMetaFieldsAndTestAvroWithMeta() { @@ -94,25 +125,11 @@ private static Stream populateMetaFieldsAndTestAvroWithMeta() { }).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 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 writer = createWriter(avroSchema, populateMetaFields); List keys = new ArrayList<>(); Map recordMap = new HashMap<>(); for (int i = 0; i < 100; i++) { @@ -134,8 +151,7 @@ public void testWriteReadHFile(boolean populateMetaFields, boolean testAvroWithM 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> records = hoodieHFileReader.readAllRecords(); records.forEach(entry -> assertEquals(entry.getSecond(), recordMap.get(entry.getFirst()))); hoodieHFileReader.close(); @@ -145,7 +161,7 @@ public void testWriteReadHFile(boolean populateMetaFields, boolean testAvroWithM Set rowsToFetch = getRandomKeys(randomRowstoFetch, keys); List rowsList = new ArrayList<>(rowsToFetch); Collections.sort(rowsList); - hoodieHFileReader = new HoodieHFileReader(conf, filePath, cacheConfig, filePath.getFileSystem(conf)); + hoodieHFileReader = (HoodieHFileReader) createReader(conf); List> result = hoodieHFileReader.readRecords(rowsList); assertEquals(result.size(), randomRowstoFetch); result.forEach(entry -> { @@ -160,6 +176,35 @@ public void testWriteReadHFile(boolean populateMetaFields, boolean testAvroWithM } } + @Override + @Test + public void testWriteReadWithEvolvedSchema() throws Exception { + // Disable the test with evolved schema for HFile since it's not supported + } + + @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 hfileReader = + new HoodieHFileReader<>(fs, new Path("dummy_base_path"), content); + Schema avroSchema = getSchemaFromResource(TestHoodieReaderWriterBase.class, "/exampleSchema.avsc"); + Iterator iterator = hfileReader.getRecordIterator(avroSchema); + + 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++; + } + } + private Set getRandomKeys(int count, List keys) { Set rowKeys = new HashSet<>(); int totalKeys = keys.size(); diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/io/storage/TestHoodieOrcReaderWriter.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/io/storage/TestHoodieOrcReaderWriter.java index 68143a215c51c..b1ad6e4225dfc 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/io/storage/TestHoodieOrcReaderWriter.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/io/storage/TestHoodieOrcReaderWriter.java @@ -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 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 @@ private HoodieOrcWriter createOrcWriter(Schema avroSchema) throws Exception { 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 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 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]); - } - - @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 hoodieReader = HoodieFileReaderFactory.getFileReader(conf, filePath); - Iterator 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++; - } + assertEquals(NUM_RECORDS, orcReader.getNumberOfRows()); } - @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,map:map>>", - reader.getSchema().toString()); - assertEquals(3, reader.getNumberOfRows()); - - HoodieFileReader hoodieReader = HoodieFileReaderFactory.getFileReader(conf, filePath); - Iterator 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++; + @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,map:map>>", + orcReader.getSchema().toString()); } } - @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 hoodieReader = HoodieFileReaderFactory.getFileReader(conf, filePath); - Schema evolvedSchema = getSchemaFromResource(TestHoodieOrcReaderWriter.class, "/exampleEvolvedSchema.avsc"); - Iterator 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: fix filterRowKeys test for ORC due to a bug in ORC logic } } diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/io/storage/TestHoodieReaderWriterBase.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/io/storage/TestHoodieReaderWriterBase.java new file mode 100644 index 0000000000000..8da0c1db4ce76 --- /dev/null +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/io/storage/TestHoodieReaderWriterBase.java @@ -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 createWriter( + Schema avroSchema, boolean populateMetaFields) throws Exception; + + protected abstract HoodieFileReader 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 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); + + HoodieFileReader hoodieReader = createReader(conf); + + Iterator iter = hoodieReader.getRecordIterator(); + int index = 0; + while (iter.hasNext()) { + GenericRecord record = iter.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++; + } + } + + @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 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); + + HoodieFileReader hoodieReader = createReader(conf); + + Iterator iter = hoodieReader.getRecordIterator(); + int index = 0; + while (iter.hasNext()) { + GenericRecord record = iter.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++; + } + } + + @Test + public void testWriteReadWithEvolvedSchema() throws Exception { + writeFileWithSimpleSchema(); + + Configuration conf = new Configuration(); + HoodieFileReader hoodieReader = createReader(conf); + String[] schemaList = new String[] { + "/exampleEvolvedSchema.avsc", "/exampleEvolvedSchemaChangeOrder.avsc", + "/exampleEvolvedSchemaColumnRequire.avsc", "/exampleEvolvedSchemaColumnType.avsc", + "/exampleEvolvedSchemaDeleteColumn.avsc"}; + + for (String evolvedSchemaPath : schemaList) { + validateReaderWithSchema(evolvedSchemaPath, hoodieReader); + } + } + + @Test + public void testReaderFilterRowKeys() throws Exception { + writeFileWithSimpleSchema(); + Configuration conf = new Configuration(); + verifyMetadata(conf); + validateFilterRowKeys(createReader(conf)); + } + + protected void writeFileWithSimpleSchema() throws Exception { + Schema avroSchema = getSchemaFromResource(TestHoodieReaderWriterBase.class, "/exampleSchema.avsc"); + HoodieFileWriter 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(); + } + + private void validateFilterRowKeys(HoodieFileReader hoodieReader) { + Set candidateRowKeys = IntStream.range(40, 100) + .mapToObj(i -> "key" + String.format("%02d", i)).collect(Collectors.toSet()); + List 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 validateReaderWithSchema(String schemaPath, HoodieFileReader hoodieReader) throws IOException { + Schema evolvedSchema = getSchemaFromResource(TestHoodieReaderWriterBase.class, schemaPath); + Iterator iter = hoodieReader.getRecordIterator(evolvedSchema); + int index = 0; + while (iter.hasNext()) { + validateRecord(schemaPath, iter.next(), index); + index++; + } + } + + private void validateRecord(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")); + } +} From 61a7ecfc46e68103a90fd55b73642e6cad884457 Mon Sep 17 00:00:00 2001 From: Y Ethan Guo Date: Fri, 18 Mar 2022 17:11:12 -0700 Subject: [PATCH 29/36] Add more HFile reader tests --- .../storage/TestHoodieHFileReaderWriter.java | 27 +++++++++++++++++++ .../storage/TestHoodieReaderWriterBase.java | 2 +- .../common/table/log/HoodieLogFileReader.java | 2 +- .../hudi/io/storage/HoodieHFileReader.java | 2 -- 4 files changed, 29 insertions(+), 4 deletions(-) diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/io/storage/TestHoodieHFileReaderWriter.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/io/storage/TestHoodieHFileReaderWriter.java index 672a5f9b7f9dc..a9c4df89cdb2b 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/io/storage/TestHoodieHFileReaderWriter.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/io/storage/TestHoodieHFileReaderWriter.java @@ -55,6 +55,8 @@ 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; @@ -205,6 +207,31 @@ public void testReadHFileFormatRecords() throws Exception { } } + @Test + public void testReaderGetRecordIterator() throws Exception { + writeFileWithSimpleSchema(); + HoodieHFileReader hfileReader = + (HoodieHFileReader) createReader(new Configuration()); + List 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 iterator = hfileReader.getRecordIterator(keys, avroSchema); + + List 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++; + } + } + private Set getRandomKeys(int count, List keys) { Set rowKeys = new HashSet<>(); int totalKeys = keys.size(); diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/io/storage/TestHoodieReaderWriterBase.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/io/storage/TestHoodieReaderWriterBase.java index 8da0c1db4ce76..10336df465bb6 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/io/storage/TestHoodieReaderWriterBase.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/io/storage/TestHoodieReaderWriterBase.java @@ -216,7 +216,7 @@ protected void writeFileWithSimpleSchema() throws Exception { } private void validateFilterRowKeys(HoodieFileReader hoodieReader) { - Set candidateRowKeys = IntStream.range(40, 100) + Set candidateRowKeys = IntStream.range(40, NUM_RECORDS * 2) .mapToObj(i -> "key" + String.format("%02d", i)).collect(Collectors.toSet()); List expectedKeys = IntStream.range(40, NUM_RECORDS) .mapToObj(i -> "key" + String.format("%02d", i)).sorted().collect(Collectors.toList()); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieLogFileReader.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieLogFileReader.java index 9555f9f08dead..347187f8301eb 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieLogFileReader.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieLogFileReader.java @@ -79,7 +79,7 @@ public class HoodieLogFileReader implements HoodieLogFormat.Reader { private long reverseLogFilePosition; private long lastReverseLogFilePosition; private boolean reverseReader; - private final boolean enableRecordLookups; + private boolean enableRecordLookups; private boolean closed = false; private transient Thread shutdownThread = null; diff --git a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileReader.java b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileReader.java index 05c78dab7aa6d..592edc105c0a4 100644 --- a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileReader.java +++ b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileReader.java @@ -239,7 +239,6 @@ public List> readRecords(List keys) throws IOException { */ public List> readRecords(List keys, Schema schema) throws IOException { this.schema = schema; - reader.getHFileInfo(); List> records = new ArrayList<>(); for (String key: keys) { Option value = getRecordByKey(key, schema); @@ -252,7 +251,6 @@ public List> readRecords(List keys, Schema schema) throw public ClosableIterator getRecordIterator(List keys, Schema schema) throws IOException { this.schema = schema; - reader.getHFileInfo(); Iterator iterator = keys.iterator(); return new ClosableIterator() { private R next; From 27ea17a5c08f3db37b05360dd67540759a50d098 Mon Sep 17 00:00:00 2001 From: Y Ethan Guo Date: Fri, 18 Mar 2022 17:16:18 -0700 Subject: [PATCH 30/36] Improve TestInLineFileSystemHFileInLining --- .../fs/inline/TestInLineFileSystemHFileInLining.java | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/hudi-common/src/test/java/org/apache/hudi/common/fs/inline/TestInLineFileSystemHFileInLining.java b/hudi-common/src/test/java/org/apache/hudi/common/fs/inline/TestInLineFileSystemHFileInLining.java index 1abbf01c1163f..6105cc01d1aa1 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/fs/inline/TestInLineFileSystemHFileInLining.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/fs/inline/TestInLineFileSystemHFileInLining.java @@ -113,8 +113,6 @@ public void testSimpleInlineFileSystem() throws IOException { FSDataInputStream fin = inlineFileSystem.open(inlinePath); HFile.Reader reader = HFile.createReader(inlineFileSystem, inlinePath, cacheConf, true, inlineConf); - // Load up the index. - reader.getHFileInfo(); // Get a scanner that caches and that does not use pread. HFileScanner scanner = reader.getScanner(true, false); // Align scanner at start of the file. @@ -195,14 +193,18 @@ private void readAndCheckbytes(HFileScanner scanner, int start, int n) int i = start; for (; i < (start + n); i++) { 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()); + 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(key, 0, key.length).getKey(); - assertArrayEquals(Arrays.copyOfRange(kv.getRowArray(), kv.getRowOffset(), kv.getRowOffset() + kv.getRowLength()), keyBytes, + 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)); From 07a5a462ecda53d76ee03662afb458b52c5e2def Mon Sep 17 00:00:00 2001 From: Y Ethan Guo Date: Fri, 18 Mar 2022 17:23:19 -0700 Subject: [PATCH 31/36] Address comments --- .../java/org/apache/hudi/testutils/HoodieClientTestUtils.java | 4 +++- .../common/fs/inline/TestInLineFileSystemHFileInLining.java | 4 +++- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestUtils.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestUtils.java index 8a3abfd6e1cbf..c1f05f9c99d85 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestUtils.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestUtils.java @@ -38,6 +38,7 @@ 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; @@ -243,7 +244,8 @@ public static Stream readHFile(JavaSparkContext jsc, String[] pat Schema schema = null; for (String path : paths) { try { - HFile.Reader reader = HFile.createReader(fs, new Path(path), cacheConfig, true, 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.getHFileInfo().get(KEY_SCHEMA.getBytes()))); } diff --git a/hudi-common/src/test/java/org/apache/hudi/common/fs/inline/TestInLineFileSystemHFileInLining.java b/hudi-common/src/test/java/org/apache/hudi/common/fs/inline/TestInLineFileSystemHFileInLining.java index 6105cc01d1aa1..190ad398e1b60 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/fs/inline/TestInLineFileSystemHFileInLining.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/fs/inline/TestInLineFileSystemHFileInLining.java @@ -19,6 +19,7 @@ package org.apache.hudi.common.fs.inline; import org.apache.hudi.common.testutils.FileSystemTestUtils; +import org.apache.hudi.io.storage.HoodieHFileUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataInputStream; @@ -112,7 +113,8 @@ public void testSimpleInlineFileSystem() throws IOException { InLineFileSystem inlineFileSystem = (InLineFileSystem) inlinePath.getFileSystem(inlineConf); FSDataInputStream fin = inlineFileSystem.open(inlinePath); - HFile.Reader reader = HFile.createReader(inlineFileSystem, inlinePath, cacheConf, true, inlineConf); + 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. From 2956986485612d21bdff9c1ac232767cc16d1154 Mon Sep 17 00:00:00 2001 From: Y Ethan Guo Date: Fri, 18 Mar 2022 17:31:11 -0700 Subject: [PATCH 32/36] Add hfile fixture and compatibility tests --- .../test/resources/hudi_0_11_hbase_2_4_9.hfile | Bin 0 -> 6115 bytes 1 file changed, 0 insertions(+), 0 deletions(-) create mode 100644 hudi-client/hudi-client-common/src/test/resources/hudi_0_11_hbase_2_4_9.hfile diff --git a/hudi-client/hudi-client-common/src/test/resources/hudi_0_11_hbase_2_4_9.hfile b/hudi-client/hudi-client-common/src/test/resources/hudi_0_11_hbase_2_4_9.hfile new file mode 100644 index 0000000000000000000000000000000000000000..2b570920f14567f6d6ba2e0c729f3ab8b923ccdb GIT binary patch literal 6115 zcmeH|`8U-2AICpq9s3dz8cUL8OvqA<%qTJ@OETpuE@76?$PyE>TvTK)#?qkD9E!g7 zOjOid#yYlYidM@-7+QwH+)wx1bGoO$;QPb8KIirRd|vO@=k+}A^LU-}dLKl4qU{f3 z4FLdB0s!JB{C`ad08jw9d0#UF3ci1s5BvAtEVNSUGRgZ^VTvv$PZz`B(c<~}|9s}@ zPId1Jo4gr&Jp1a*OG3kRbY#;rM9VYRmqlfE7hH~h-eUc4dn|x9@e$5Y?uqHNruX*RD%JwCg zBn_CdMsQjwSvmAzl>T-Dx93SY)@jHIz9esBtQXJYI zkWYVrg2EmkCu|%qzu(6iiPusZ%%O{;^l1iea!KyUi%44``(-$#IME$Rpg4nB2bv_5 z8F7s+3aKf2B}z8UHPm5WgcRUNNFi)HXz$Y z?3e4hM~uMW4zgVg3|J%TVDJFhZtIx(-hwq^qos5&ht5Ok`x>}uCc3w!A>+XsL&`Ui zl=l|BBLpz*Kzj&O#MG>OR)#Wz^XX*p-v`KeGChba!!r>FfWS|!_gozQ>0JN-h=A>p zWB<}Sk?lWw7j@hwTgg;<6Rl(pKCXz;IAX@Q#N|rI7&oIxCncjzTg>`^ECRo$=*8y8 z)uyF4atsKH6okC6UOXRd4&z4ee>;QZo~@--2*-WjB!pU-ocAG5s&glMTQmRHAX3dm z%^!kRbXUI;_H=^4JxhzyyJ?-f!;5dX`JTj2H1X7YdrHvcxaN$9^#!-^aO=8*sXHG@ zOJqF3MRgc|VeOSSTdpd!4uzHnTYC8&3tt^VSYTm}rd_w{9|K|q&~Fth!pa&JRzYUw zF`XI}hWGltW?8xo&4-UoDN>0o6_Wb#WtxwO&a4Vp_8)5%gAHFOd{3r`)F+*RJVS*8 z*?xoCX_uyxV-!8oOB9--w)cQ{8`opMDn|3db=%x+65l9tX}W@v*)ZIQre+P?{UBST;dGec!7P4OVko zOXCf^=`}BIMBH4AswgvgRSiepc=N!0;flS~D6%`C1czl9*(Dvfa&oqpUSo$S`uj}1+#hufhwCbpcttUwWU$M2 z(e!rD6>aP0bqr)S+=VDEmrr#1OASo!nL=AC2Zu^tIHPJ&dkR> zN0dp0Jx7zMYwpT#4PVHNW~lfztx|88Z`5wtfw1k>xx%kxT&x(xuAEz+$?jtJPM^=5 zbFk{Yh*4Z#KeQlZ<8t=CgV)8-9$@Qpc)#L)g!?^;x#|H+=Dc2RZ2FPJSRM<{xpF0ksp?3V!nIQ<+D z0e*>)KW&qx?{ccHiHS*n;cF*~0vko{4luCr`nfI3|IsoAw^Ts~O?2Ylivn=|uNDAm zNlU}i$1(fX)>_XL?b})90C>QQEq|1`wew;fF^3&6u8!czAXw0ce>oY1_){(3&Bd(~ zr36>VqzoO%fL#%f8hjeY)oDr3AQ3wnD|6-R4T5qrH=IxC$Y^isFKr6(^B3zA&oBE} zlJT$mK7R0dw9zU~C_l$djA3xPo<3;~_{{Le9l0ub64gIkb}7}kYk@<(FZ<48!JQ+0 z5*@0chZ)8DqxZBNCK(mE%PflfOqh8;4TgT_TuuG#&4WF)Fs*Am{!Zhx-POJ~c|c*9 z4yzzDs*s>wpwBhZ=&19gGCrjHR@d|GdrMACEnr=hUEk@iY_*ih3mLAc)DsP+G85i8 z3hy@z%MstBw)QeM=(6Od=IB9=ciNo(*By0Rw_&YL(7q)l{?(c7@%kVvBD{yVmvtoK z{0oU#*{maqP%RGm3U#JwR&{#ll=8UxEDH6N8$$dDQegryN1)oLC>WY@2k4nqLo z>nYh3PmdorypFC5lqW4eA*Bk_TJ+F|m_YWLp&(9xJK$|b4ksY|!~6q?K?q`0APGSt zBO)Tn;r@gmgcF$@ND6YY_m2p|9y>q|4f7B82UC8aw!j1g1Ox;G1Ox;G1Ox;G1Ox;G V1Ox;G1Ox;G1O)y+2nYj!@ZTQb!+-z) literal 0 HcmV?d00001 From 5327266035df97fbb4c97a0f4fd2f036ce508a3b Mon Sep 17 00:00:00 2001 From: Y Ethan Guo Date: Sat, 19 Mar 2022 21:29:30 -0700 Subject: [PATCH 33/36] Add HFile compatibility tests --- .../storage/TestHoodieHFileReaderWriter.java | 86 +++++++++++++--- .../storage/TestHoodieReaderWriterBase.java | 94 +++++++++--------- ...ase_1_2_3_bootstrap_index_partitions.hfile | Bin 0 -> 19745 bytes .../hudi_0_10_hbase_1_2_3_complex.hfile | Bin 0 -> 6612 bytes .../hudi_0_10_hbase_1_2_3_simple.hfile | Bin 0 -> 6115 bytes ...ase_2_4_9_bootstrap_index_partitions.hfile | Bin 0 -> 19745 bytes .../hudi_0_11_hbase_2_4_9_complex.hfile | Bin 0 -> 6612 bytes ...ile => hudi_0_11_hbase_2_4_9_simple.hfile} | Bin ...ase_1_2_3_bootstrap_index_partitions.hfile | Bin 0 -> 19745 bytes .../hudi_0_9_hbase_1_2_3_complex.hfile | Bin 0 -> 6612 bytes .../hudi_0_9_hbase_1_2_3_simple.hfile | Bin 0 -> 6115 bytes 11 files changed, 120 insertions(+), 60 deletions(-) create mode 100644 hudi-client/hudi-client-common/src/test/resources/hudi_0_10_hbase_1_2_3_bootstrap_index_partitions.hfile create mode 100644 hudi-client/hudi-client-common/src/test/resources/hudi_0_10_hbase_1_2_3_complex.hfile create mode 100644 hudi-client/hudi-client-common/src/test/resources/hudi_0_10_hbase_1_2_3_simple.hfile create mode 100644 hudi-client/hudi-client-common/src/test/resources/hudi_0_11_hbase_2_4_9_bootstrap_index_partitions.hfile create mode 100644 hudi-client/hudi-client-common/src/test/resources/hudi_0_11_hbase_2_4_9_complex.hfile rename hudi-client/hudi-client-common/src/test/resources/{hudi_0_11_hbase_2_4_9.hfile => hudi_0_11_hbase_2_4_9_simple.hfile} (100%) create mode 100644 hudi-client/hudi-client-common/src/test/resources/hudi_0_9_hbase_1_2_3_bootstrap_index_partitions.hfile create mode 100644 hudi-client/hudi-client-common/src/test/resources/hudi_0_9_hbase_1_2_3_complex.hfile create mode 100644 hudi-client/hudi-client-common/src/test/resources/hudi_0_9_hbase_1_2_3_simple.hfile diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/io/storage/TestHoodieHFileReaderWriter.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/io/storage/TestHoodieHFileReaderWriter.java index a9c4df89cdb2b..899de4d61d266 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/io/storage/TestHoodieHFileReaderWriter.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/io/storage/TestHoodieHFileReaderWriter.java @@ -18,6 +18,7 @@ package org.apache.hudi.io.storage; +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; @@ -35,6 +36,7 @@ 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.io.hfile.CacheConfig; import org.apache.hadoop.hbase.io.hfile.HFile; import org.apache.hadoop.hbase.util.Pair; @@ -42,6 +44,7 @@ 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.IOException; @@ -69,6 +72,12 @@ import static org.mockito.Mockito.when; 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"; @Override protected Path getFilePath() { @@ -80,7 +89,7 @@ protected HoodieFileWriter createWriter( Schema avroSchema, boolean populateMetaFields) throws Exception { String instantTime = "000"; HoodieWriteConfig writeConfig = HoodieWriteConfig.newBuilder() - .withPath("dummy_base_path") + .withPath(DUMMY_BASE_PATH) .withIndexConfig(HoodieIndexConfig.newBuilder() .bloomFilterNumEntries(1000).bloomFilterFPP(0.00001).build()) .withPopulateMetaFields(populateMetaFields) @@ -192,19 +201,10 @@ public void testReadHFileFormatRecords() throws Exception { fs.open(getFilePath()), (int) fs.getFileStatus(getFilePath()).getLen()); // Reading byte array in HFile format, without actual file path HoodieHFileReader hfileReader = - new HoodieHFileReader<>(fs, new Path("dummy_base_path"), content); + new HoodieHFileReader<>(fs, new Path(DUMMY_BASE_PATH), content); Schema avroSchema = getSchemaFromResource(TestHoodieReaderWriterBase.class, "/exampleSchema.avsc"); - Iterator iterator = hfileReader.getRecordIterator(avroSchema); - - 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++; - } + assertEquals(NUM_RECORDS, hfileReader.getTotalRecords()); + verifySimpleRecords(hfileReader.getRecordIterator(avroSchema)); } @Test @@ -232,6 +232,44 @@ public void testReaderGetRecordIterator() throws Exception { } } + @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 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 getRandomKeys(int count, List keys) { Set rowKeys = new HashSet<>(); int totalKeys = keys.size(); @@ -243,4 +281,26 @@ private Set getRandomKeys(int count, List keys) { } 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()); + } } diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/io/storage/TestHoodieReaderWriterBase.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/io/storage/TestHoodieReaderWriterBase.java index 10336df465bb6..19f9b9385165a 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/io/storage/TestHoodieReaderWriterBase.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/io/storage/TestHoodieReaderWriterBase.java @@ -108,19 +108,7 @@ public void testWriteReadPrimitiveRecord() throws Exception { Configuration conf = new Configuration(); verifyMetadata(conf); verifySchema(conf, schemaPath); - - HoodieFileReader hoodieReader = createReader(conf); - - Iterator iter = hoodieReader.getRecordIterator(); - int index = 0; - while (iter.hasNext()) { - GenericRecord record = iter.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++; - } + verifySimpleRecords(createReader(conf).getRecordIterator()); } @Test @@ -147,34 +135,7 @@ public void testWriteReadComplexRecord() throws Exception { Configuration conf = new Configuration(); verifyMetadata(conf); verifySchema(conf, schemaPath); - - HoodieFileReader hoodieReader = createReader(conf); - - Iterator iter = hoodieReader.getRecordIterator(); - int index = 0; - while (iter.hasNext()) { - GenericRecord record = iter.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++; - } + verifyComplexRecords(createReader(conf).getRecordIterator()); } @Test @@ -189,7 +150,7 @@ public void testWriteReadWithEvolvedSchema() throws Exception { "/exampleEvolvedSchemaDeleteColumn.avsc"}; for (String evolvedSchemaPath : schemaList) { - validateReaderWithSchema(evolvedSchemaPath, hoodieReader); + verifyReaderWithSchema(evolvedSchemaPath, hoodieReader); } } @@ -198,7 +159,7 @@ public void testReaderFilterRowKeys() throws Exception { writeFileWithSimpleSchema(); Configuration conf = new Configuration(); verifyMetadata(conf); - validateFilterRowKeys(createReader(conf)); + verifyFilterRowKeys(createReader(conf)); } protected void writeFileWithSimpleSchema() throws Exception { @@ -215,7 +176,46 @@ protected void writeFileWithSimpleSchema() throws Exception { writer.close(); } - private void validateFilterRowKeys(HoodieFileReader hoodieReader) { + protected void verifySimpleRecords(Iterator 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 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 hoodieReader) { Set candidateRowKeys = IntStream.range(40, NUM_RECORDS * 2) .mapToObj(i -> "key" + String.format("%02d", i)).collect(Collectors.toSet()); List expectedKeys = IntStream.range(40, NUM_RECORDS) @@ -224,17 +224,17 @@ private void validateFilterRowKeys(HoodieFileReader hoodieReader) .stream().sorted().collect(Collectors.toList())); } - private void validateReaderWithSchema(String schemaPath, HoodieFileReader hoodieReader) throws IOException { + private void verifyReaderWithSchema(String schemaPath, HoodieFileReader hoodieReader) throws IOException { Schema evolvedSchema = getSchemaFromResource(TestHoodieReaderWriterBase.class, schemaPath); Iterator iter = hoodieReader.getRecordIterator(evolvedSchema); int index = 0; while (iter.hasNext()) { - validateRecord(schemaPath, iter.next(), index); + verifyRecord(schemaPath, iter.next(), index); index++; } } - private void validateRecord(String schemaPath, GenericRecord record, int 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()); diff --git a/hudi-client/hudi-client-common/src/test/resources/hudi_0_10_hbase_1_2_3_bootstrap_index_partitions.hfile b/hudi-client/hudi-client-common/src/test/resources/hudi_0_10_hbase_1_2_3_bootstrap_index_partitions.hfile new file mode 100644 index 0000000000000000000000000000000000000000..91e9c7656c39ef755da0f779bd4e64ff19eeddec GIT binary patch literal 19745 zcmeHOZHQb~8J?ZQ8WybD7Hdhdbu6Sx&d&L~=bYF=H_fiQ%_bYN+n6?l^?W9i&hD%; zv(dB!i%=f2PTabSFYeE-bB&r~Y<0(@Th&+1XFR350n53j@5?n>q6$;yY9 zO}l$P#T3^us}oiE%Id8c`C4c5AHTPKQPF?@v5Y>z-Q=YRk6vrIfAx_7cw^}lTwFiYFDw9S@#>}XhSHk`>- zZgVHOvTQSSuYhpP`C<6Vze#-Z{6}Y%D#tssGitYHvT>q`xQ6r@g68p0Awy z!JhBF_QRh%^v)ZveCj)wZ+f=++#S{GZR@Z8tIwx=795pSAiO$_p=)e2bn3l7q`p2xU5&wu@{22>TYVA^XuPv`GE#l( zdv8DakA>*&Pzf$FZqvxg_DBh_8GvC#jnOj&H0WYQze14cXw zgRJHnG0U9eDnu-4GCVyrcoB0EqE*OB;t^K@fclsrMTD>cw!H6!ruKtYPJGW;GVc@YvAc8I1|!L8CxehPdId_EaX2rPwkoS9+5g9X{`SQm@A%P+158tIV483~SE*;So($B#I(R z6u|Qt@N|$FG7M`kfpo?jklygfh+^JE9|Q`(EP-h>35$UQ?NJf%i_nJJh%%0ttufI}e&q9j$Ha?p5EZ~x9z7I_E z&c~lR_~!j*-gxWlkLdj~DOiT3NldV$ zq+UqnlHq9xr!xV`iz0(qHRpmMi?M?o)eNlc;5A7NdjY&cDwhn8VPm2Td6W=>OOV0H z5OB;4LA)Ardovbwu!&5Q4A0u1qLh0cjYq7fxk@Mm>Bf)=szE~xjtX=)8D1fkOO{>+ zJY6cUk@xw=0^|}I1xn&E2vSoWL^9JP#V)L5cn17e3UZ)ShKs)!s=iY&0&dPMoNBf7qw0oP1&d>4>Uf$5z&MU2;AoS8|WoYG-2g0j6 zC#Pp7rVmfTh}B=f=cR!Wt8X3I|M29`J}~F<9XDZC|oe z{ll;RmDl0y8f-jz$LL27PCT-3Y<%Y7iG^bmN9U$z58slnq><6PMsGbdethB3#G$$Q z*`pH+^K%Pu>n*#kEVI1r>AZqd;$mxQ4C>SRMb2Z5e8rAU<#n@Hui|_9<mc1WE~%5-255N}!a$dn$n){{_Er Bq>%ss literal 0 HcmV?d00001 diff --git a/hudi-client/hudi-client-common/src/test/resources/hudi_0_10_hbase_1_2_3_complex.hfile b/hudi-client/hudi-client-common/src/test/resources/hudi_0_10_hbase_1_2_3_complex.hfile new file mode 100644 index 0000000000000000000000000000000000000000..8ce3d0d0b1f6c35d938ad3ca939a3a2f2d59f3e0 GIT binary patch literal 6612 zcmeH}`#)6qAIHxi2^oak8a1_&%eb=*iy<0`CdRFdTxVR;XlfQU$)#vrBDP#|8?}U( zkZZ~%mLX%*AjPKEWoKA}vW!cP=k^ZA@}424Bm zp0ra306}X25MSQ-rw9Om1pqess3eMj_sxR4iz{<+ma#ET&$^*3(J68#Rph5w>-IdA zRSnBymWM14!QpVM>E1$D%NW=7FF9OyXmCe0ZSr%Qyld>D!<);&*+o6gP4Ka+5z~ncp%WPwfMTS$;5oe5$!&GI^!iJ>*`W zH9``TjDNywk~OG~kv4GBZ}>sRus}yGNft|O$nMFdK7!n|`uMm&Qj2BzLOwweBP7!* z8r1KoH8G1}MZu|P7tHP2M_LOLV9h7m$9sotVA(tMbGxl~tGTGTxL|&+EeRnJ)q*T4 zQI(Z3jZbkQ8MOF9VHpz{6Ad+c6P84SGAOEy3C(pNuRJ#ES+Dp$xhRyy_9%ABy}(iZ zAOQ+tiZ6lD#)@CIWaJPVTU>l)FyGqsTUDE5_k0O=T2SC-1m^`X61o`26<>do=J9To}8;s6km z51p{L-dbh803ZY|Bq;Mgt4wI$_9`R0nxx3+iho7P7=VWZB8L)?II`=fUu?opXPlp-6=EBUzrcCE9R^(9yDXU)zA;vma zaKv$?x$&Nc`JrSAo?Xow~@Co7>VuT7hjUDV|1fgi0|QuOS;k0MfP z6JJ!_dx+OWRvx?gU9US;LGvK5=`g#~y7 zX!qi6QWe`4pUzA&k~`{9UKW@jbX8dHOPNsP?xIg$pD*i5<2&Da!U>^%8L2AfUGlLE z;d>%0)CECVWkX|sJ%jKzWvV)@*Nsg-R9$=0WkNbLppGi4b}e6}Ip74H3QLLQQTwVF zBQ~%dLSmocj5KvAs}!%k197(|W5T5!6YfjZgzb^eO5hRq<#u58#2o&ZH1zH0nl_eL zVk`dCF{0TR(|taA$M2!TnrT`^yAw$_RhG|hD6bQlrWB(nTphlga^#F=z~@s_0|))? z_Xn3}+w^yh{U(X{ZiY}j-C^ic@k*AV`Agi>k^);;j8A4?1nQp6@PVK!+_uQSgnV~} zW%LhZo*j8Ek4z1fYN}C37N5Rj`%25P!s(ETj~_7yY90Y;oxb(v#ojh14&B~op&rKD zQxYea#x$hg*0>%RGrXdok3YhK7o0Fx`wbaT;u6_)Wo-Y_}^A=-_>rbGJWP?LQBQMC_ zy^I&p(wkc@km5+C!XL~!q$@-*)P0`!utx5mG-}Y>UySy3J~Qc(KfZwyG$A+8Lgb!` z*3Ay<^{?-nJFuiV>vdt8GK9z%3mrlQX7VtyW9n~rvl8XJYL+t#4c06697O!k15euc zr7@n`XSeX<+Ehv>vuE;3GRNAuC*D?ie(m_50w(qs>#g15L%M;zLz=zPW(W)|!a&~2 zsAF37&eep|QCFAf_~E47%iW`C`F5%bYn4s#@dBrwaPFJp>t%VQ0VWgxN?$_FZI8J* zo;roq19t}v@Pz$mg8B|JF8Iw2?+x3VGGA^?P~GL2wh;#%SC-hXLn zflcz@36`Dl|9sMc6aUr%KrLZmIqTVnIcsYkb+2;LSa5(%4~3c4SHZXa?KiU6IDsmPRb$FBUE#u5sKGY zaX2a3O-I#+BNUqX$vuY=HutsZ&hdIK(j?eUL&pAW!^M~0QfK^P2%1q7lD3B4ET}Xt zE#+wEM=NI3Teudd*`eh8Oi1E5>Gp8aa>7@qIfqbuL0YL*8;EW!TL_m0ldiw_WBLc{NfrvmN;?iXz$(!JhKog06 zf&OSq?@Ru6E>^^lP;Zhq81jN_y+!;Bp8%f#p8%f#p8%f#p8%f#p8%f#p8%f#p8%i0 J{|5oV{{iwrx_1Bo literal 0 HcmV?d00001 diff --git a/hudi-client/hudi-client-common/src/test/resources/hudi_0_10_hbase_1_2_3_simple.hfile b/hudi-client/hudi-client-common/src/test/resources/hudi_0_10_hbase_1_2_3_simple.hfile new file mode 100644 index 0000000000000000000000000000000000000000..abe0b336eb3c22fd46dc31b69f6cb5a67dda5def GIT binary patch literal 6115 zcmeH|`8U-2AICpq9s3dz8cUL8OvqA<%qTJ@OETpuE@76?$PyE>TvTK)#?qkD9E!g7 zOjOid#yYlYidM@-7+QwH+)wx1bGoO$;QPb8KIirRd|vO@=k+}A^LU-}dLKl4qU{f3 z4FLdB0s!JB{C`ad08jw9d0#UF3ci1s5BvAtEVNSUGRgZ^VTvv$PZz`B(c<~}|9s}@ zPId1Jo4gr&Jp1a*OG3kRbY#;rM9VYRmqlfE7hH~h-eUc4dn|x9@e$5Y?uqHNruX*RD%JwCg zBn_CdMsQjwSvmAzl>T-Dx93SY)@jHIz9esBtQXJYI zkWYVrg2EmkCu|%qzu(6iiPusZ%%O{;^l1iea!KyUi%44``(-$#IME$Rpg4nB2bv_5 z8F7s+3aKf2B}z8UHPm5WgcRUNNFi)HXz$Y z?3e4hM~uMW4zgVg3|J%TVDJFhZtIx(-hwq^qos5&ht5Ok`x>}uCc3w!A>+XsL&`Ui zl=l|BBLpz*Kzj&O#MG>OR)#Wz^XX*p-v`KeGChba!!r>FfWS|!_gozQ>0JN-h=A>p zWB<}Sk?lWw7j@hwTgg;<6Rl(pKCXz;IAX@Q#N|rI7&oIxCncjzTg>`^ECRo$=*8y8 z)uyF4atsKH6okC6UOXRd4&z4ee>;QZo~@--2*-WjB!pU-ocAG5s&glMTQmRHAX3dm z%^!kRbXUI;_H=^4JxhzyyJ?-f!;5dX`JTj2H1X7YdrHvcxaN$9^#!-^aO=8*sXHG@ zOJqF3MRgc|VeOSSTdpd!4uzHnTYC8&3tt^VSYTm}rd_w{9|K|q&~Fth!pa&JRzYUw zF`XI}hWGltW?8xo&4-UoDN>0o6_Wb#WtxwO&a4Vp_8)5%gAHFOd{3r`)F+*RJVS*8 z*?xoCX_uyxV-!8oOB9--w)cQ{8`opMDn|3db=%x+65l9tX}W@v*)ZIQre+P?{UBST;dGec!7P4OVko zOXCf^=`}BIMBH4AswgvgRSiepc=N!0;flS~D6%`C1czl9*(Dvfa&oqpUSo$S`uj}1+#hufhwCbpcttUwWU$M2 z(e!rD6>aP0bqr)S+=VDEmrr#1OASo!nL=AC2Zu^tIHPJ&dkR> zN0dp0Jx7zMYwpT#4PVHNW~lfztx|88Z`5wtfw1k>xx%kxT&x(xuAEz+$?jtJPM^=5 zbFk{Yh*4Z#KeQlZ<8t=CgV)8-9$@Qpc)#L)g!?^;x#|H+=Dc2RZ2FPJSRM<{xpF0ksp?3V!nIQ<+D z0e*>)KW&qx?{ccHiHS*n;cF*~0vko{4luCr`nfI3|IsoAw^Ts~O?2Ylivn=|uNDAm zNlU}i$1(fX)>_XL?b})90C>QQEq|1`wew;fF^3&6u8!czAXw0ce>oY1_){(3&Bd(~ zr36>VqzoO%fL#%f8hjeY)oDr3AQ3wnD|6-R4T5qrH=IxC$Y^isFKr6(^B3zA&oBE} zlJT$mK7R0dw9zU~C_l$djA3xPo<3;~_{{Le9l0ub64gIkb}7}kYk@<(FZ<48!JQ+0 z5*@0chZ)8DqxZBNCK(mE%PflfOqh8;4TgT_TuuG#&4WF)Fs*Am{!Zhx-POJ~c|c*9 z4yzzDs*s>wpwBhZ=&19gGCrjHR@d|GdrMACEnr=hUEk@iY_*ih3mLAc)DsP+G85i8 z3hy@z%MstBw)QeM=(6Od=IB9=ciNo(*By0Rw_&YL(7q)l{?(c7@%kVvBD{yVmvtoK z{0oU#*{maqP%RGm3U#JwR&{#ll=8UxEDH6N8$$dDQegryN1)oLC>WY@2k4nqLo z>nYh3PmdorypFC5lqW4eA*Bk_TJ+F|m_YWLp&(9xJK$|b4ksY|!~6q?K?q`0APGSt zBO)Tn;r@gmgcF$@ND6YY_m2p|9y>q|4f7B82UC8aw!j1g1Ox;G1Ox;G1Ox;G1Ox;G T1Ox;G1Ox;G1O)zX2nhcT9@N8q literal 0 HcmV?d00001 diff --git a/hudi-client/hudi-client-common/src/test/resources/hudi_0_11_hbase_2_4_9_bootstrap_index_partitions.hfile b/hudi-client/hudi-client-common/src/test/resources/hudi_0_11_hbase_2_4_9_bootstrap_index_partitions.hfile new file mode 100644 index 0000000000000000000000000000000000000000..7f6c5bd353d224e535f8d012330af914f88ccc64 GIT binary patch literal 19745 zcmeI3UudLP6~HI6Ya1fAx-DW|&~|8Lk-eGw@4J7t7MkrQCTy|^n{-`UmU#berk%{h znaSFuihg z4Vz?kzVF_9=G=4dcYf#Gb7Ep?;`r>z&sHk>MfiE)-OX3EQhB%nAG{2|4p%C7OjYh& zGo9W;6jNNstWH$rYn!(ooVQPmR6lMmblT%xw-Q#(JD+;G*1NbCYR77w;MyHuJ6da* z)%^d{?Y3`(#&@4ZL3o<16dFF}^X# zYTJh<+!!07>ASUKk3W5__Y0xZh0fqwfB461jaD!3a=qECojY3dA)57OuXc=FI=a=! z?hEOb za=Sap7t20FZ*;KO`08#ux}nstWE*jF+@%KW1ludLzq)SNgK;i&yYH>-TIj4cx&u4I zz^dNc#HJP(cD5assdYnV!tFT+TTAafS={!fcD~bIU%SO)w0kSJ{CjkJ4fo{5u0EH-zDm>dhcFbZ8lm@?&^Q@sk*Tr?pnX+E~P!G&RwcJ z@cO%d>HPipC-0c}PU}ClJ0Gk5_?~L@?(Ika_3x*Ju!@Jsk+e2LJZ218=?!u!M#ZD? ziit|K3jbogx4KsEw%0o@)bq#va?jXi=+-y>koxuzbv+Ib$}g{nUiBGxLF0oPZ$_#= z{mE-Dy|r9@>)Ai&f1uObCT3@=qo*e)PEU(WkGKsjW>%7B?O48QcZ3l)0DqT( z{=boC@0)KtKYDTffv48~efjy{yJrVTQ|};6ct1^^fb5`#k+gFrY&p%_4? zG(6Gc(#53tv%W+4&zp7#70;{)P9UuXMcitPG~))b=oJO`RdbTcO1fSMBgx_kFO~Nm zgwnb86c@+{ED+(~`Fo-=LUgkB3Q03rJWv-V0YKOhP#nxMP#n!D3QpL{&q$>J5l zNV0e=vXDiXS4J7Q(}5bKB>bX;`^+c^BqWojt{0+QvUm{2$TI3Z(i8(}@=PO@K_rF) zrel*~pM`YWD`cf)7?D7{#PEQWbFef-U`R_5A(nudP(H&?n1>=@eo_45nFSQF&&Bk-{dg+@XPRW$+O%2cEidIAwj7HJsiA^jrqy^-a zr~?8el0oLiB#T$*uO>T$B2q>J)5sDB+5=T0r4&QbJ5RJ^8R3bf3EC^n2>-m#>HoTE zPAA~igNiVcEN3MPnL`sGxs+v)>@-Y791=+r#4AL(Wbts80gQvXq6Q-28gNrla!=cl4ePoC27LB^rJWA#Pam)&hz@~<>^^CBECeWBJ`h#x|kS5tdn(%%)W z-%$9NI;vgjDKbimQxEChKAc4jdA5~iIm&s=!JP*GP)6XS5IFTZ6@*E#Lzpa{i4>08 z35PNU=i4BW2^l64Fv5_1HQ-kp_><`ElX{`IpDdmx1hT#wGSosrPBVj~h+&YVQ#M+a z2?0cxigKhljF2oImjFqMz?5pp*>DNCsg!}^pqND-qb0$GWbGAl`Y-#O{?5&Fy7AaL zNC8LU0p&P%kg$exw3;|but%emO`qICZ$DXk(E*O+Swi7eo=?{nqD5eUaEumgQ2L-^ zDl3ut@HpL5mPJn`f*e2CUXTEp0hB2S98zFj6T=cyuCTEbl4i2@d=wJS;#(Ahft4#M zSfRCWw1YZ`w!w*7uG7UEUJynIr#Q2~z*q_hBS3J?1`LO36$B9gHz||pE4MJpk*dAQ z;6tYJ*Xq~jW=@rW-cv53esv+WHp%W3t0_g6RdH z7rb?QW;HL*yndh?QbTKf^<=o1*Jutj+_TIN%6{hm@wbN__#=7!`#)T`^vdt(kE&;2 z@BiHXB|Fvcee2b{4rg?*@z_0MA3Hhu#pN>-vyVpWRWw`aw=ruFT+pgpltm6}BKA&G&o}KKsJv?@J?5^!g{mwTp-Z3?H z@7Ue_i;qA#&=Pb!Gnef)(JvVY>1Xt9-hexI>3zf&_28)zkdv&$l8i)F{ev$KdBe&S`>AY_CdMm!a zUk-ip%p>hpD0ecwcIQj5zRFjrfl>pd21*T-8YneTYM|6Wsew`hr3Ok3lo}{C@cwDw IK&5ivzcv7@!~g&Q literal 0 HcmV?d00001 diff --git a/hudi-client/hudi-client-common/src/test/resources/hudi_0_11_hbase_2_4_9_complex.hfile b/hudi-client/hudi-client-common/src/test/resources/hudi_0_11_hbase_2_4_9_complex.hfile new file mode 100644 index 0000000000000000000000000000000000000000..f5293c5a249dad32cd766eeecf75b4f2644b29f9 GIT binary patch literal 6612 zcmeI1`#)6qAIHxi2^oak8a1_&%eb=*iy<0`CdRFdTxVR;XlfQU$)#vrBDP#|8?}U( zkZZ~%mLX%*AjPKEWoKA}vW!czT2VQ3R!Tj;5=7!1Sm1_5p zdwtdjNlY^S39m`kpgKm{z)8R12N}Zx9knD`EVUuKCztvNa?|SL;{r)7mgNii1VxOH zOsi;6zoXW~EQS>Yr=ndjw`(71ElhwdpJ*TN9kPLC@6^xjw%)DgqUPd)`MI_vghW&e zvZzE=R>m|w#f47cJZx*=>GID@Tg)OXu*AyTlhuGNS;wyvs)~?^0+8n#*OSsd5 zqJIm$D5Wj>g?&?6DS9zb_Fndz)Uy@##~IK?fx{Fsm-EGHJyj(RwD{;v1ET=xJD54q zxoJv-rQkup0#^CUrQ^y(?Pg9IN}xhj~H(ajCc{-=l+*lKe$paaL; zqrr@Y#~Ka~j!2VLhqADQ>>@gCE1I<#WoILrH|b*tq+DAvJ|J$*YzrEltU8cYlC{ve z1#K%4k)619bURo!seXMsbF!#`{$X3TA$WFyvSl!R`edm-wAae>?{Q1LIli~UB4Jq^ z07CMi6ZY0yv&A6d7IduP7MDiAM@tVlWV>iER7T=Zl(wW?;>@%E^rY>cb;?;K`?$%^XxU^%!eW{wTJM-!yl7|z8ziD z#u7_x#h*GxG#g{O&qwe0J#<(zO{-{kBI%~e^7#$rbt2Q0Vibj|!AohnbGG_sVr&x4MnEZm{<&_IgOwinU(F#C&R0MBz{t z?xqzb_>ni{9y)2z=%U?))uy34K=1vRw>RXF6H>jO&+X_ZIXBK=5Gn&LepB2HJ5NLB z;>Ye5D(mm*CjLOxtM})X>9tLDPYdz7cirYLQ8?l9J33;P6`hIeEokn;f&Mj#_|nSg zrzNw2{ijuxiEc_&`!BI>H{fEr4C=cjxX!l-NtR+P(=Olp&UTF&=FzUk=o6OWJmz_6 zZ`{N?GxA+$=gyp6?<0;xBr@|4=c#IQvJ|?OS{cdJk0H|Y1*2`=VvKS936zm+aENH+ z1=+io@giD!bIS!%9H~_JgIR}kg(!x)&+{JE$o-Q>4SM^F(Z0@SCSCH!H&B8m(mNaL*E=*H~5cy)EL#V(^9!7Rd{q1g6qMTRFa%Q2ydc~fDh#z|3 zNgKa3##8(37JgitO6g?wOkPRmSR41m+e*)`9sg6n#QtKvwOf2hH?Vg|vsc;-fuThh z$Xgk8Osn3xns7Sm>JlA4oRoXHdo(TIPE}#8vI#z3;M5b&eN%kBERQt6gaSb6OQ^Z+ zF*nCkr?7h9>YxEu*nbuX0KL6H_^elv~o@ywu&?MZ7} z2OH2CBn{A}5A|<37k6{DbqTxXz@;#i>_4 zp=u`s)ym9Er7G>K9vHkcjEX6Rsm((&3Uqubf?bb@i5O?VO6uiKw_a;5QK%8++_hX~ z-_RqKFTyMhgZ_AJ24z@$+?mExniKVi^4;a~t`VOwlPx8OzG)SDNMU!mZ`|{R6SFrp zi5KqHX-8Cz2`Rq|?^pHBI7_a=es|!E)6`NURzOAJ2{)y<{a4Oa{0_m3DsFM6g{|<* zPxdh&#h#`-7%X0+s-^aDHo6ix{IgC^{6uTe1Iv+X)^MAsN}6_34xt{QniG}q!Mt9- z`uL5*NzraPsx};<(9BQnIgGHmuT6K3*K?63!FC!l_GcR|zVwzl;}=8FjGB{`SF{9qXwK&ZVCFf^C630omhm)2|hTNeKxJs-rh;}b{hi@);wqPAMKk49y z>_w=OlJQV^*#1f{HHZ%6dp!#X8o&T&^oS$^!aLO4H^3heK=unn1QHRKE)z-K1b+mY zNc0QzM_YPd^0#xbB8G%|lf1!@7i8-x;_vtb_yqU__yqU__yqU__yqU__yqU__yqU_ O_yqnB2nd4zZ2kxGO}csj literal 0 HcmV?d00001 diff --git a/hudi-client/hudi-client-common/src/test/resources/hudi_0_11_hbase_2_4_9.hfile b/hudi-client/hudi-client-common/src/test/resources/hudi_0_11_hbase_2_4_9_simple.hfile similarity index 100% rename from hudi-client/hudi-client-common/src/test/resources/hudi_0_11_hbase_2_4_9.hfile rename to hudi-client/hudi-client-common/src/test/resources/hudi_0_11_hbase_2_4_9_simple.hfile diff --git a/hudi-client/hudi-client-common/src/test/resources/hudi_0_9_hbase_1_2_3_bootstrap_index_partitions.hfile b/hudi-client/hudi-client-common/src/test/resources/hudi_0_9_hbase_1_2_3_bootstrap_index_partitions.hfile new file mode 100644 index 0000000000000000000000000000000000000000..290af9918e5dc78f12fc6bb82d2a692e529f4224 GIT binary patch literal 19745 zcmeHOTWnlM8D4Kfu`E%bjSwXglvNQ?mGRE?%uIk%r?IgWCys14N{Ave*JRVR*WF#a zgb?U6NIdk3OL+rI4aR23n?14y6>8VMmp`OmK7%i71ulx-`d zv+}Ob?m08_oqsOp|Nd|O6BCOQ$7fGISgGh|;r-p$H(u3B<>3na@C*1jT&dhWReATS zX?H(CF~xPv>O@t(xN+&hIeTTK`aZMXZjE=`<*;o2^}%bk?$y;$J63B4*J}IP(b|ex z&c9E$TD}pEw_4p!w{2F>K%I?lqqTA>bd5J%vvFl-)wulfR{PSpf#b_z{PLP_jQ1K@ zZu!uJ3u7ZReW!Np!nIqquZMOADuZ+V<}cJ5E8V=xwPv$+@o3G5Xx5tD+A(td=w>0i z3(OhQy}YfCUTL^~oTB*;sjOSN$7L)!urzYyMump7x}=c)jwK z3okss^uXDPOE0szn?HVLvihz2s?|eVkN%y<)6x;~l zQmw*Ytaq1J>z&qG+l6}mxL@iT+YFt0?;ojeHBndN@SuErEp)3-!V3x?>>Z3$|90b7 zUwUn+`rNmFn*W1J@0ysMt&Yx5PMoNYo}N2BSskg4=FUR@vof8zXdN{S$;?W`brz6P zil9iDrZIX?olJ(Oi{QQ5hxffZh8MZ^L>hr$fdlJ#8<7bzAe?y;snpbRn+%T^!ArIt z58R6go$H<8x%Dj4%roSLHB4mONR=hSlS6|iFjdA8g;)j{IpYpl9xZZ0kl?k{T#ICQ zxCq`~`knji9e1vd1T!K9qD;WpqfsCweLzMiO1)-Gk(dmR6v0dGT!XP>QFG|h9G+Eh zWSvG9bHXSUSb3Zbj}8r9796tzCIpczAg;iUS_qiFL`8!c7%}Q(cx-6!1fy8vpb?V> zriKk1S*IML)^M$aGnq@bo){XuOavlvP_U8^0%H#=WF<8yN(a52QDzxI+Od4rZVMy! z2>z}C`v1l>pYxAD_`-i4(#NhH`txr_zWmJrrm43vO*ozomKqhoA=Fp^R|=6L?N~%u zI>{q5A>%I5A=LFkmnNCY1*yHa!Giy6fs6plL`G#9(gC=LRE8>>$~1Mo(49_(hqWeD zx`?6&uaJwLzp63v86}!oBQZ`TBf4JbuO`C_#zm=tWP+&xo>oAU@@@u3Oa#{4=`5AX z>H6?^I=2=^7mT>fQtn@fJXw>3yvV0#}YvFiOMrfNMOnE3S&gc@NmqqhZmwU zf(aSKu`Hz)MFZ=LC)`Gn-noTLGZ~%-7VJFdNJa@%P7n$l7|<$VMN~m)qIk0P3Zop! z@Gu9pj=;$=5G1ld zEfMEhf^b7L0>y^XK%8({B9#MpgG{=&PzlL1Amog9w;%v$a?r$2ZxnT^j861 zp+lGqFM_1WeI`NEcP zL(fCz=65Uy?qZ~EqMDv<$AGAR61CgFR39cfHyQETp4`G6f#XxGR=}{mQ1r`ny@eZ=xupoxqEi|e*Lv__bhA? z-zP@1wF|gpnr{Qs{J@_kumAGjzyHQd7jJz2pC@lVJ-{@{Hl_*3(@qg>ne!-s`v>kc zIE03Ai9ld7p69-yNaadoh-8FO7i5Nk11s3t#tAsB8azSdtfG-q8&v8pGD?Q%5W*Hb zp)>{=E(6__8^u8O0s#y#N+B4wKBiJRQZMxOll@g{u>k5O5XmT*(8ESmYeB*RJ&6*N z8B0~-9z3w#@WKc#WUvJkbl?aZKw$@C26{TN86w1B?*izqETjoH0B?9q6I4uvB_+U> zGSD<5c>Bf(f|!y@aBXeCDj8m3l;gF25ax$>9E5?C6e2R%R;fika~broAt1SWu&EZn zmV`-^T)}#UR4y5w3#*BfonHq#4!wX?m z1|Kq&U(a8AV&=rT`MJ483bgYmypIedb5>?@KH7ij7Z2}~(fKj{&YRmRJ9w)UB!s@W zkr`U~@QLuVgHtoJlQXBMpvCI*@czj_i`7@o9Dj7`NADPW^}yKJ%;^)8=a%5=xqJR_ zU{}uQlXq?IUD`VpYcrE^X2plk&aA{1c+PoN>D!VkdfarvM*iW>w4}|u_#ed%*AAu| ze4g;u>6+y{J#%M6w`C10Ys)9Y)jUUYpx~ZqevtOVude^#kLRC$d3Nr~M;@B~=z8@m zEUq8lKV_%-)JwfQoYBF;WA}}{_vGaHrLz;Wk4`R~ot$5onLB+?K9WYpJ~Vdksflw- zrzTG=EY8hOE-fxB!KL?%-Wq0k*~@tb>-fakPvxhUW+(e)504!lyLanUzw(WfcTbJo zKX#~p@)1Y}T7-&cPUW?4tNG~IyZTqmPAn|ujSfx&x-B&G`VIYs#rZrxy7oY0^hWnR zFLvI4VmvLts7DF@=O^jP>NiId}60zly+o&}m zCghrOiDk$bHAu0kby>q2lyw{V9(Etu$NmN1ALer&pL5RVywCgfd3~Ped_LzKLt#-? zC+#%=K+pyN#8)=|DFOgs34j-URgy%&<5t1l!IinX$k>`>WM5a7=oGn~Ch}99O?$q| zx~A1JD^)91I2?{O+gIdf73;R~IhX4J4e6++Pkw5XcZ*wce0?b-r?_W2!^BNxGIDLX z`LF5T8SUzm@G1Il&!Vm^Rm|DFMC`^Bv!BMjZOB{TTABNkmAK9=H8zec;rL zFgGQ9GHxX;O(~DAL@_kaI`z@n$^1{s=vegvgJ;EqCymU4jm%$lUeep?Roxz7O0#iP zK-8hA33r}ZB{i}_w+?4m(7B?-%)B3<}R_-qxpK5NHOj)b;2))~9 zgOJ3e;2*Cx$r@J2N*g*GG<+{(RH&<-EQ_T!$6KXsO+uA*}XP<)m_zHT`@n`mV}as>cN(k zsLION#wWPYOnO3*u#BmUsiwMvDO;jJ85C8Mrm3*S4X{rEGID4dOQT&LEUC~78Lzk z=y@r9$v^z7(pvHJfwFh9U!|U|O@EjLT@*S_A@ew&tvAwC;z5fKUpFudk$!_&6P;V8 zMA!k!0Tp;TSCk+WE>}%jb04`M;?QWn-MQH zkV~sEJj*JhC=b3LqW02JbvP-fTJGY^2 zCn9o^ct>}FWs@4$b}}c68yN3*WE(=}7Ae~XGiFYf8bEukz5X7z)a&E>IxG`c!~r0* z06O7dvpviF06++wNKn>)W|`3bomob9GfkDz6aRveF$6b9L@p&TX=Kk&zt~2c(!7M0 z(fsng9giu2z#C-0`7*mw!+j5Jgjfazm__k&X2K0&^Wo-WQ)crQD)Om*1o*Nln8 z^0eYl9V1$evEAolcKseUtd*`^yf=w-LuKXMrt$`nWkxZM#?|4=DM!v|1%5g;HE_uP zUVlh=j%|O}*l&`EZ)XYBGaW|06)$C(TEE0UDJisr#rkITMWXK74j&A@%x#PMOUQ3e zcxL}V)(<1kV=N>&(s9&-b;laOn0vOO5c= zZ%X3j(pg4~TbkE`Vu#lZ3h;+nPPKQ-^O?6L-@OpRL=Pyz?5%Rk_VpkMhh#M_v?t_7ToSv0aAs-4a}vn}lR5F}7Kk-#r)mMoo(tHxu*;EAds2R_Yx|Um+Db>72K~r)AJyhv9KXjP}V9&c1+{VUUrh4cg;#xk>N(gH-`}4_rQ}k ze`!ph_SrA~$eT*-Wc5s5PT|;?^d#6xFYu24DPZbwq29(lA+#IVH>A}oZH~auBMs%P zjXP%4ZeK|}9erh)fgeuJyVO0JUSO}Lz^iP6j~6=kL~vizu9fAJ23Sx4D18C7usi1N zbm|mVA6y+Y!3z7&0s)|R76?D@Xvm}?N_cRVeL_$$er;2-MF0q2XBor5L|)O{zJFK(7Z4PTLyH87YQB1N z^?{Pe!S-w8qi4tZUY*~6QTU0e7I&Ren;JgKAxPxLQgGb+`GbFO9@6SeYh0asH4>|K zGf}Oq{4}c4{_26j+ry~XQkePzB(qT0w<5&th?s~;Cak1h?sV(b<`RV(QO+H!_37*S zr1B-0l~M2?ugsxL%MZKLSC!^PJ)`~hxV~+~C(hOhh8FSs zojRS!sxcwuw-NnnewjaztFYf5JmWmI+=vxWQFzQvrM3Ucxq{y%ctOQI-mIt$Q)xT>89 zRZ=nmDi1qQ>8%dYg?y)P2|)uG;0%IDA|QOieEb3f5P?4aL}C~s(AVc;0Kz^X($(i2 zIY1RWu_lIw`H*}r5lP;V?KgkE literal 0 HcmV?d00001 diff --git a/hudi-client/hudi-client-common/src/test/resources/hudi_0_9_hbase_1_2_3_simple.hfile b/hudi-client/hudi-client-common/src/test/resources/hudi_0_9_hbase_1_2_3_simple.hfile new file mode 100644 index 0000000000000000000000000000000000000000..e52d3c556e555032233725fc948226c05c3e60db GIT binary patch literal 6115 zcmeH|`8U-2AICpq9s3dz8cWh(OvqA<%qTLZm}JUTTwOU8hOs-0 zGXMZcF#w1g^Zzv=06+ua;(g6@DER+jo}Ax%GBHXi%RwHu3X*lPxw=>;j~>U*`)8V~ zJK431GI2BJMAp^mmx1+DQ4x*Lkj>AWUlx|yUN~~>`^NO*%*4{dG3&8-J?|Tb=8Prs zY>K_bdjGn4Jw=~3JKEpgYaCv>5gDRFtJpm+)GJy4wb3(~>)rIQ5oc`O`Oi=;qIBnGryi;Tekqfd8kKdq?d5shuAHh=Aph zM1++XTsZdoW?-nL@p(`3g!P4!S z&~t~N;gxwq=2)B~Lv@Uwb?y#cUtzM^y!B%cV>fgwgV*Jxn`Ozk+)y9w6+*dWSR1Blv$;7xy z+<$SNBY%5($6M~L12>G^lE+Vd&qY+y9_4!9sg@{GYds<+v=%7W{UtN3ZU#%t<6N;& z+2>q+Cn!xMMoi`N9TBF??f08)t0s!X&KF)fffWCOY)#uP)j#^h@}b|#SA>-{%q;^< z&7wOr$_?)IxzDh5>zfWApH!rgn#(2OaiyA%NKWi>Sk@nFfY8ZKy26!jgPx15`}D16TX>`NfV4Uq$>Rq6!8_X(z|pRIRcp3ayO> zR*a)L*j^*onq#_}zW(GAIa?~EeeTlm!>t`0PsgVNXoIM=EtRP%xg2wLreSRwQ%M{fj_hHFJ+nBAJM+Eu zc@EH5{3MgFVl}DI_vv@h!7u{zBOa+a`q+PZk&p8-Cot~*hSuD8#kLLr>Uv*=Xt0}F zn;UKrO|H2wg~!fDtBSIcR@DgPjmM6Qw?+roWiM9MCN@+q1Pvb5(;&O6*J*{1WVU!F zb(!7omR%%V3cO+`HG=B$D<@EBwkVVsp*{3 zU39(OvxVEbc-nP%VpyoK2rS?dnPd!%E6(M7tSbOiy%UcOqW*66lN8&b#5*= z=WR>qH}Gxi85t`+Y|&z>#c=c^cuXBHo#11Hts`6Rg{m}2HZO2Z2RF7aY4Z>Z-e=}w zpCe19D90_ja4YB5r?WaaJyYj1 zX6-F|E@Bmz*ALALSsyuj-`@RVXg9F+IigQ-KN5eBYNmR?f;Fd?6O(o{A%@2$a<5#E z=pN54#p!KZuWi+tC_B~@y*SK>ze5ij<;Vd*)e|{e>_K-|7Z*2Eusaxn4*R7+0FFN! zgr9c;AT#@Yhq&3U-;SyqQFKW-W~=fUO%^G`9E4l;F2onpoxzE`%nN*|J4FO zEoo^)+8B1<+FHw*!hJg{?EzOrk;RWRmo{FE1NN{z*4Y8P83YIV&@VTGkbbJgxg2rn zKr0~>GN^+G(qUJ`BL|*RmUNoa(t}9t4HY@^b@~B085>R~b!4{Je#_g!9Xu z7G&b<-j5$VA8oXV2bP^%N{D81JD)yj^83v6ARN6ac?#V(RC+1JsB@l6yD$6Bbso=^ zK7|R@(8G=peK32P4+j|*;$;@ZJ;zNwo(4m|bFZd+_Ta&unw!)%oOq{k+V*Pin_QrP zqQlP5h%5+H&xbD=YP8q7(U>37ysGN>c0I)>C+Bg_%FgfLD_bpOazlp7EA&KzX{`8n z4#N8lDB0qB)Ye|c1YDNf)Eqg;^+=tCf89~LbsNs|B>h`r!e1R(uCEWm!o#{rd)Y_B z?~i%qAgDW0sd?&eO4L~cB*L2@$v;cuSEFGWK~2Y`71E^!{b8x32DR#k5ZU$XB|{JZ z`06X0?B@F8h1b!=LV42i<5H?Htp!&B*I6b*3%@PG%B92SW5q4@Zb0+1vhe=?bZ zB>DP;2Ox0))Z;!Okpa5k Date: Sat, 19 Mar 2022 21:38:59 -0700 Subject: [PATCH 34/36] Exclude org.apache.hadoop.hbase.KeyValue.KeyComparator from shading for backward compatibility --- packaging/hudi-flink-bundle/pom.xml | 3 +++ packaging/hudi-hadoop-mr-bundle/pom.xml | 3 +++ packaging/hudi-integ-test-bundle/pom.xml | 3 +++ packaging/hudi-kafka-connect-bundle/pom.xml | 3 +++ packaging/hudi-presto-bundle/pom.xml | 3 +++ packaging/hudi-spark-bundle/pom.xml | 3 +++ packaging/hudi-timeline-server-bundle/pom.xml | 3 +++ packaging/hudi-trino-bundle/pom.xml | 3 +++ packaging/hudi-utilities-bundle/pom.xml | 3 +++ 9 files changed, 27 insertions(+) diff --git a/packaging/hudi-flink-bundle/pom.xml b/packaging/hudi-flink-bundle/pom.xml index 68767b656b8d4..4d0b18c67ed74 100644 --- a/packaging/hudi-flink-bundle/pom.xml +++ b/packaging/hudi-flink-bundle/pom.xml @@ -179,6 +179,9 @@ org.apache.hadoop.hbase. org.apache.hudi.org.apache.hadoop.hbase. + + org.apache.hadoop.hbase.KeyValue$KeyComparator + org.apache.hbase. diff --git a/packaging/hudi-hadoop-mr-bundle/pom.xml b/packaging/hudi-hadoop-mr-bundle/pom.xml index 7115fbd0c4bef..337afc168d5cd 100644 --- a/packaging/hudi-hadoop-mr-bundle/pom.xml +++ b/packaging/hudi-hadoop-mr-bundle/pom.xml @@ -118,6 +118,9 @@ org.apache.hadoop.hbase. org.apache.hudi.org.apache.hadoop.hbase. + + org.apache.hadoop.hbase.KeyValue$KeyComparator + org.apache.hbase. diff --git a/packaging/hudi-integ-test-bundle/pom.xml b/packaging/hudi-integ-test-bundle/pom.xml index fdab05db1164d..78e76a32dd432 100644 --- a/packaging/hudi-integ-test-bundle/pom.xml +++ b/packaging/hudi-integ-test-bundle/pom.xml @@ -199,6 +199,9 @@ org.apache.hadoop.hbase. org.apache.hudi.org.apache.hadoop.hbase. + + org.apache.hadoop.hbase.KeyValue$KeyComparator + org.apache.hbase. diff --git a/packaging/hudi-kafka-connect-bundle/pom.xml b/packaging/hudi-kafka-connect-bundle/pom.xml index 9e131cc76e517..d17b99b6820d6 100644 --- a/packaging/hudi-kafka-connect-bundle/pom.xml +++ b/packaging/hudi-kafka-connect-bundle/pom.xml @@ -161,6 +161,9 @@ org.apache.hadoop.hbase. org.apache.hudi.org.apache.hadoop.hbase. + + org.apache.hadoop.hbase.KeyValue$KeyComparator + org.apache.hbase. diff --git a/packaging/hudi-presto-bundle/pom.xml b/packaging/hudi-presto-bundle/pom.xml index b50c79fe33063..ad6d2ec98e157 100644 --- a/packaging/hudi-presto-bundle/pom.xml +++ b/packaging/hudi-presto-bundle/pom.xml @@ -109,6 +109,9 @@ org.apache.hadoop.hbase. org.apache.hudi.org.apache.hadoop.hbase. + + org.apache.hadoop.hbase.KeyValue$KeyComparator + org.apache.hbase. diff --git a/packaging/hudi-spark-bundle/pom.xml b/packaging/hudi-spark-bundle/pom.xml index 7e6f01ede94d0..fd79a3460f82e 100644 --- a/packaging/hudi-spark-bundle/pom.xml +++ b/packaging/hudi-spark-bundle/pom.xml @@ -150,6 +150,9 @@ org.apache.hadoop.hbase. org.apache.hudi.org.apache.hadoop.hbase. + + org.apache.hadoop.hbase.KeyValue$KeyComparator + org.apache.hbase. diff --git a/packaging/hudi-timeline-server-bundle/pom.xml b/packaging/hudi-timeline-server-bundle/pom.xml index 99664399c3407..e1e5d9854fb84 100644 --- a/packaging/hudi-timeline-server-bundle/pom.xml +++ b/packaging/hudi-timeline-server-bundle/pom.xml @@ -227,6 +227,9 @@ org.apache.hadoop.hbase. org.apache.hudi.org.apache.hadoop.hbase. + + org.apache.hadoop.hbase.KeyValue$KeyComparator + org.apache.hbase. diff --git a/packaging/hudi-trino-bundle/pom.xml b/packaging/hudi-trino-bundle/pom.xml index 68861cb6e8a0c..7cbd502dbe600 100644 --- a/packaging/hudi-trino-bundle/pom.xml +++ b/packaging/hudi-trino-bundle/pom.xml @@ -111,6 +111,9 @@ org.apache.hadoop.hbase. org.apache.hudi.org.apache.hadoop.hbase. + + org.apache.hadoop.hbase.KeyValue$KeyComparator + org.apache.hbase. diff --git a/packaging/hudi-utilities-bundle/pom.xml b/packaging/hudi-utilities-bundle/pom.xml index 2771992f3f6cf..c46a6d7d61e35 100644 --- a/packaging/hudi-utilities-bundle/pom.xml +++ b/packaging/hudi-utilities-bundle/pom.xml @@ -187,6 +187,9 @@ org.apache.hadoop.hbase. org.apache.hudi.org.apache.hadoop.hbase. + + org.apache.hadoop.hbase.KeyValue$KeyComparator + org.apache.hbase. From c916edb98791835386c0e50cf632fecd9ad72812 Mon Sep 17 00:00:00 2001 From: Y Ethan Guo Date: Mon, 21 Mar 2022 11:53:37 -0700 Subject: [PATCH 35/36] Improve docs --- .../org/apache/hudi/io/storage/HoodieHFileReader.java | 4 ++-- .../org/apache/hudi/io/storage/HoodieHFileUtils.java | 10 +++++----- 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileReader.java b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileReader.java index 592edc105c0a4..90440345f7f4b 100644 --- a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileReader.java +++ b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileReader.java @@ -92,8 +92,8 @@ public HoodieHFileReader(Configuration configuration, Path path, CacheConfig cac this.reader = HoodieHFileUtils.createHFileReader(fs, path, cacheConfig, configuration); } - public HoodieHFileReader(FileSystem fs, Path path, byte[] content) throws IOException { - this.reader = HoodieHFileUtils.createHFileReader(fs, path, content); + public HoodieHFileReader(FileSystem fs, Path dummyPath, byte[] content) throws IOException { + this.reader = HoodieHFileUtils.createHFileReader(fs, dummyPath, content); } @Override diff --git a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileUtils.java b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileUtils.java index ebacf2977e752..3767ea1832579 100644 --- a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileUtils.java @@ -59,20 +59,20 @@ public static HFile.Reader createHFileReader( /** * Creates HFile reader for byte array with default `primaryReplicaReader` as true. * - * @param fs File system. - * @param path Dummy path to file to read. - * @param content Content in byte array. + * @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 path, byte[] content) throws IOException { + 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(path) + .withFilePath(dummyPath) .withInputStreamWrapper(stream) .withFileSize(content.length) .withFileSystem(fs) From 0733dcc3ad7c7912a59375fa524939728beb8c35 Mon Sep 17 00:00:00 2001 From: Y Ethan Guo Date: Tue, 22 Mar 2022 11:35:43 -0700 Subject: [PATCH 36/36] Address minor comments --- .../io/storage/TestHoodieHFileReaderWriter.java | 1 + .../io/storage/TestHoodieOrcReaderWriter.java | 2 +- .../java/org/apache/hudi/integ/ITTestBase.java | 16 +++------------- 3 files changed, 5 insertions(+), 14 deletions(-) diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/io/storage/TestHoodieHFileReaderWriter.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/io/storage/TestHoodieHFileReaderWriter.java index 899de4d61d266..e1f97949ef102 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/io/storage/TestHoodieHFileReaderWriter.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/io/storage/TestHoodieHFileReaderWriter.java @@ -191,6 +191,7 @@ public void testWriteReadHFileWithMetaFields(boolean populateMetaFields, boolean @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 diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/io/storage/TestHoodieOrcReaderWriter.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/io/storage/TestHoodieOrcReaderWriter.java index b1ad6e4225dfc..282f102697d78 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/io/storage/TestHoodieOrcReaderWriter.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/io/storage/TestHoodieOrcReaderWriter.java @@ -95,6 +95,6 @@ protected void verifySchema(Configuration conf, String schemaPath) throws IOExce @Override public void testReaderFilterRowKeys() { - // TODO: fix filterRowKeys test for ORC due to a bug in ORC logic + // TODO(HUDI-3682): fix filterRowKeys test for ORC due to a bug in ORC logic } } diff --git a/hudi-integ-test/src/test/java/org/apache/hudi/integ/ITTestBase.java b/hudi-integ-test/src/test/java/org/apache/hudi/integ/ITTestBase.java index f2a5395af32ea..7ec2ba506101b 100644 --- a/hudi-integ-test/src/test/java/org/apache/hudi/integ/ITTestBase.java +++ b/hudi-integ-test/src/test/java/org/apache/hudi/integ/ITTestBase.java @@ -221,8 +221,6 @@ private TestExecStartResultCallback executeCommandInDocker(String containerName, // Each execution of command(s) in docker should not be more than 15 mins. Otherwise, it is deemed stuck. We will // try to capture stdout and stderr of the stuck process. - LOG.error("containerName: " + containerName); - LOG.error("Command: " + Arrays.asList(command)); boolean completed = dockerClient.execStartCmd(createCmdResponse.getId()).withDetach(false).withTty(false).exec(callback) .awaitCompletion(540, SECONDS); @@ -238,11 +236,8 @@ private TestExecStartResultCallback executeCommandInDocker(String containerName, int exitCode = dockerClient.inspectExecCmd(createCmdResponse.getId()).exec().getExitCode(); LOG.info("Exit code for command : " + exitCode); if (exitCode != 0) { - //LOG.error("\n\n ###### Stdout #######\n" + callback.getStdout().toString()); + LOG.error("\n\n ###### Stdout #######\n" + callback.getStdout().toString()); } - callback.getStderr().flush(); - callback.getStdout().flush(); - LOG.error("\n\n ###### Stdout #######\n" + callback.getStdout().toString()); LOG.error("\n\n ###### Stderr #######\n" + callback.getStderr().toString()); if (checkIfSucceed) { @@ -343,8 +338,8 @@ private void saveUpLogs() { executeCommandStringInDocker(HIVESERVER, "cat /tmp/root/hive.log | grep -i exception -A 10 -B 5", false).getStdout().toString(); String filePath = System.getProperty("java.io.tmpdir") + "/" + System.currentTimeMillis() + "-hive.log"; FileIOUtils.writeStringToFile(hiveLogStr, filePath); - LOG.error("Hive log saved up at : " + filePath); - LOG.error("<=========== Full hive log ===============>\n" + LOG.info("Hive log saved up at : " + filePath); + LOG.info("<=========== Full hive log ===============>\n" + "\n" + hiveLogStr + "\n <==========================================>"); } catch (Exception e) { @@ -361,11 +356,6 @@ void assertStdOutContains(Pair stdOutErr, String expectedOutput, String stdOutSingleSpaced = singleSpace(stdOutErr.getLeft()).replaceAll(" ", ""); expectedOutput = singleSpace(expectedOutput).replaceAll(" ", ""); - LOG.error("stdOutErr : " + stdOutErr.getLeft()); - LOG.error("stdOutErr.getRight : " + stdOutErr.getRight()); - LOG.error("stdOutSingleSpaced : " + stdOutSingleSpaced); - LOG.error("expectedOutput : " + expectedOutput); - int lastIndex = 0; int count = 0; while (lastIndex != -1) {