1
0

[HUDI-3760] Adding capability to fetch Metadata Records by prefix (#5208)

- Adding capability to fetch Metadata Records by key prefix so that Data Skipping could fetch only Column Stats 
- Index records pertaining to the columns being queried by, instead of reading out whole Index.
- Fixed usages of HFileScanner in HFileReader. few code paths uses cached scanner if available. Other code paths uses its own HFileScanner w/ positional read. 

Brief change log
- Rebasing ColumnStatsIndexSupport to rely on HoodieBackedTableMetadata in lieu of reading t/h Spark DS
- Adding methods enabling key-prefix lookups to HoodiFileReader, HoodieHFileReader
- Wiring key-prefix lookup t/h LogRecordScanner impls
- Cleaning up HoodieHFileReader impl

Co-authored-by: sivabalan <n.siva.b@gmail.com>
Co-authored-by: Sagar Sumit <sagarsumit09@gmail.com>
This commit is contained in:
Alexey Kudinkin
2022-04-06 09:11:08 -07:00
committed by GitHub
parent 7612549bcc
commit 9e87d164b3
46 changed files with 1387 additions and 698 deletions

View File

@@ -339,7 +339,7 @@ public class HoodieTimelineArchiver<T extends HoodieAvroPayload, I, K, O> {
// Read the avro blocks
while (reader.hasNext()) {
HoodieAvroDataBlock blk = (HoodieAvroDataBlock) reader.next();
blk.getRecordItr().forEachRemaining(records::add);
blk.getRecordIterator().forEachRemaining(records::add);
if (records.size() >= this.config.getCommitArchivalBatchSize()) {
writeToFile(wrapperSchema, records);
}

View File

@@ -107,7 +107,7 @@ public class HoodieHFileWriter<T extends HoodieRecordPayload, R extends IndexedR
.withFileContext(context)
.create();
writer.appendFileInfo(HoodieHFileReader.KEY_SCHEMA.getBytes(), schema.toString().getBytes());
writer.appendFileInfo(HoodieHFileReader.SCHEMA_KEY.getBytes(), schema.toString().getBytes());
}
@Override

View File

@@ -18,17 +18,6 @@
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;
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;
@@ -39,7 +28,17 @@ 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;
import org.apache.hudi.common.bootstrap.index.HFileBootstrapIndex;
import org.apache.hudi.common.engine.TaskContextSupplier;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.EmptyHoodieRecordPayload;
import org.apache.hudi.common.model.HoodieAvroRecord;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.util.FileIOUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.config.HoodieIndexConfig;
import org.apache.hudi.config.HoodieWriteConfig;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.params.ParameterizedTest;
import org.junit.jupiter.params.provider.Arguments;
@@ -51,21 +50,25 @@ import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.Spliterator;
import java.util.Spliterators;
import java.util.TreeMap;
import java.util.function.Supplier;
import java.util.stream.Collectors;
import java.util.stream.IntStream;
import java.util.stream.Stream;
import java.util.stream.StreamSupport;
import static org.apache.hudi.common.testutils.FileSystemTestUtils.RANDOM;
import static org.apache.hudi.common.testutils.SchemaTestUtil.getSchemaFromResource;
import static org.apache.hudi.common.util.CollectionUtils.toStream;
import static org.apache.hudi.io.storage.HoodieHFileConfig.HFILE_COMPARATOR;
import static org.apache.hudi.io.storage.HoodieHFileReader.KEY_SCHEMA;
import static org.apache.hudi.io.storage.HoodieHFileReader.SCHEMA_KEY;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertNotNull;
import static org.junit.jupiter.api.Assertions.assertNull;
@@ -124,7 +127,7 @@ public class TestHoodieHFileReaderWriter extends TestHoodieReaderWriterBase {
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()))));
new Schema.Parser().parse(new String(hfileReader.getHFileInfo().get(SCHEMA_KEY.getBytes()))));
}
private static Stream<Arguments> populateMetaFieldsAndTestAvroWithMeta() {
@@ -142,7 +145,7 @@ public class TestHoodieHFileReaderWriter extends TestHoodieReaderWriterBase {
Schema avroSchema = getSchemaFromResource(TestHoodieOrcReaderWriter.class, "/exampleSchemaWithMetaFields.avsc");
HoodieFileWriter<GenericRecord> writer = createWriter(avroSchema, populateMetaFields);
List<String> keys = new ArrayList<>();
Map<String, GenericRecord> recordMap = new HashMap<>();
Map<String, GenericRecord> recordMap = new TreeMap<>();
for (int i = 0; i < 100; i++) {
GenericRecord record = new GenericData.Record(avroSchema);
String key = String.format("%s%04d", "key", i);
@@ -163,24 +166,30 @@ public class TestHoodieHFileReaderWriter extends TestHoodieReaderWriterBase {
Configuration conf = new Configuration();
HoodieHFileReader hoodieHFileReader = (HoodieHFileReader) createReader(conf);
List<Pair<String, IndexedRecord>> records = hoodieHFileReader.readAllRecords();
records.forEach(entry -> assertEquals(entry.getSecond(), recordMap.get(entry.getFirst())));
List<IndexedRecord> records = HoodieHFileReader.readAllRecords(hoodieHFileReader);
assertEquals(new ArrayList<>(recordMap.values()), records);
hoodieHFileReader.close();
for (int i = 0; i < 2; i++) {
int randomRowstoFetch = 5 + RANDOM.nextInt(10);
Set<String> rowsToFetch = getRandomKeys(randomRowstoFetch, keys);
List<String> rowsList = new ArrayList<>(rowsToFetch);
Collections.sort(rowsList);
hoodieHFileReader = (HoodieHFileReader) createReader(conf);
List<Pair<String, GenericRecord>> result = hoodieHFileReader.readRecords(rowsList);
assertEquals(result.size(), randomRowstoFetch);
List<GenericRecord> expectedRecords = rowsList.stream().map(recordMap::get).collect(Collectors.toList());
hoodieHFileReader = (HoodieHFileReader<GenericRecord>) createReader(conf);
List<GenericRecord> result = HoodieHFileReader.readRecords(hoodieHFileReader, rowsList);
assertEquals(expectedRecords, result);
result.forEach(entry -> {
assertEquals(entry.getSecond(), recordMap.get(entry.getFirst()));
if (populateMetaFields && testAvroWithMeta) {
assertNotNull(entry.getSecond().get(HoodieRecord.RECORD_KEY_METADATA_FIELD));
assertNotNull(entry.get(HoodieRecord.RECORD_KEY_METADATA_FIELD));
} else {
assertNull(entry.getSecond().get(HoodieRecord.RECORD_KEY_METADATA_FIELD));
assertNull(entry.get(HoodieRecord.RECORD_KEY_METADATA_FIELD));
}
});
hoodieHFileReader.close();
@@ -202,7 +211,7 @@ public class TestHoodieHFileReaderWriter extends TestHoodieReaderWriterBase {
fs.open(getFilePath()), (int) fs.getFileStatus(getFilePath()).getLen());
// Reading byte array in HFile format, without actual file path
HoodieHFileReader<GenericRecord> hfileReader =
new HoodieHFileReader<>(fs, new Path(DUMMY_BASE_PATH), content);
new HoodieHFileReader<>(fs, new Path(DUMMY_BASE_PATH), content, Option.empty());
Schema avroSchema = getSchemaFromResource(TestHoodieReaderWriterBase.class, "/exampleSchema.avsc");
assertEquals(NUM_RECORDS, hfileReader.getTotalRecords());
verifySimpleRecords(hfileReader.getRecordIterator(avroSchema));
@@ -217,7 +226,7 @@ public class TestHoodieHFileReaderWriter extends TestHoodieReaderWriterBase {
IntStream.concat(IntStream.range(40, NUM_RECORDS * 2), IntStream.range(10, 20))
.mapToObj(i -> "key" + String.format("%02d", i)).collect(Collectors.toList());
Schema avroSchema = getSchemaFromResource(TestHoodieReaderWriterBase.class, "/exampleSchema.avsc");
Iterator<GenericRecord> iterator = hfileReader.getRecordIterator(keys, avroSchema);
Iterator<GenericRecord> iterator = hfileReader.getRecordsByKeysIterator(keys, avroSchema);
List<Integer> expectedIds =
IntStream.concat(IntStream.range(40, NUM_RECORDS), IntStream.range(10, 20))
@@ -233,6 +242,59 @@ public class TestHoodieHFileReaderWriter extends TestHoodieReaderWriterBase {
}
}
@Test
public void testReaderGetRecordIteratorByKeyPrefixes() throws Exception {
writeFileWithSimpleSchema();
HoodieHFileReader<GenericRecord> hfileReader =
(HoodieHFileReader<GenericRecord>) createReader(new Configuration());
Schema avroSchema = getSchemaFromResource(TestHoodieReaderWriterBase.class, "/exampleSchema.avsc");
List<String> keyPrefixes = Collections.singletonList("key");
Iterator<GenericRecord> iterator =
hfileReader.getRecordsByKeyPrefixIterator(keyPrefixes, avroSchema);
List<GenericRecord> recordsByPrefix = toStream(iterator).collect(Collectors.toList());
List<GenericRecord> allRecords = toStream(hfileReader.getRecordIterator()).collect(Collectors.toList());
assertEquals(allRecords, recordsByPrefix);
// filter for "key1" : entries from key10 to key19 should be matched
List<GenericRecord> expectedKey1s = allRecords.stream().filter(entry -> (entry.get("_row_key").toString()).contains("key1")).collect(Collectors.toList());
iterator =
hfileReader.getRecordsByKeyPrefixIterator(Collections.singletonList("key1"), avroSchema);
recordsByPrefix =
StreamSupport.stream(Spliterators.spliteratorUnknownSize(iterator, Spliterator.ORDERED), false)
.collect(Collectors.toList());
assertEquals(expectedKey1s, recordsByPrefix);
// exact match
List<GenericRecord> expectedKey25 = allRecords.stream().filter(entry -> (entry.get("_row_key").toString()).contains("key25")).collect(Collectors.toList());
iterator =
hfileReader.getRecordsByKeyPrefixIterator(Collections.singletonList("key25"), avroSchema);
recordsByPrefix =
StreamSupport.stream(Spliterators.spliteratorUnknownSize(iterator, Spliterator.ORDERED), false)
.collect(Collectors.toList());
assertEquals(expectedKey25, recordsByPrefix);
// no match. key prefix is beyond entries in file.
iterator =
hfileReader.getRecordsByKeyPrefixIterator(Collections.singletonList("key99"), avroSchema);
recordsByPrefix =
StreamSupport.stream(Spliterators.spliteratorUnknownSize(iterator, Spliterator.ORDERED), false)
.collect(Collectors.toList());
assertEquals(Collections.emptyList(), recordsByPrefix);
// no match. but keyPrefix is in between the entries found in file.
iterator =
hfileReader.getRecordsByKeyPrefixIterator(Collections.singletonList("key1234"), avroSchema);
recordsByPrefix =
StreamSupport.stream(Spliterators.spliteratorUnknownSize(iterator, Spliterator.ORDERED), false)
.collect(Collectors.toList());
assertEquals(Collections.emptyList(), recordsByPrefix);
}
@ParameterizedTest
@ValueSource(strings = {
"/hudi_0_9_hbase_1_2_3", "/hudi_0_10_hbase_1_2_3", "/hudi_0_11_hbase_2_4_9"})
@@ -253,7 +315,7 @@ public class TestHoodieHFileReaderWriter extends TestHoodieReaderWriterBase {
HoodieHFileUtils.createHFileReader(fs, new Path(DUMMY_BASE_PATH), content),
hfilePrefix, true, HFILE_COMPARATOR.getClass(), NUM_RECORDS_FIXTURE);
HoodieHFileReader<GenericRecord> hfileReader =
new HoodieHFileReader<>(fs, new Path(DUMMY_BASE_PATH), content);
new HoodieHFileReader<>(fs, new Path(DUMMY_BASE_PATH), content, Option.empty());
Schema avroSchema = getSchemaFromResource(TestHoodieReaderWriterBase.class, "/exampleSchema.avsc");
assertEquals(NUM_RECORDS_FIXTURE, hfileReader.getTotalRecords());
verifySimpleRecords(hfileReader.getRecordIterator(avroSchema));
@@ -261,7 +323,7 @@ public class TestHoodieHFileReaderWriter extends TestHoodieReaderWriterBase {
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);
hfileReader = new HoodieHFileReader<>(fs, new Path(DUMMY_BASE_PATH), content, Option.empty());
avroSchema = getSchemaFromResource(TestHoodieReaderWriterBase.class, "/exampleSchemaWithUDT.avsc");
assertEquals(NUM_RECORDS_FIXTURE, hfileReader.getTotalRecords());
verifySimpleRecords(hfileReader.getRecordIterator(avroSchema));

View File

@@ -38,6 +38,7 @@ import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.TreeSet;
import java.util.stream.Collectors;
import java.util.stream.IntStream;
@@ -217,7 +218,7 @@ public abstract class TestHoodieReaderWriterBase {
private void verifyFilterRowKeys(HoodieFileReader<GenericRecord> hoodieReader) {
Set<String> candidateRowKeys = IntStream.range(40, NUM_RECORDS * 2)
.mapToObj(i -> "key" + String.format("%02d", i)).collect(Collectors.toSet());
.mapToObj(i -> "key" + String.format("%02d", i)).collect(Collectors.toCollection(TreeSet::new));
List<String> expectedKeys = IntStream.range(40, NUM_RECORDS)
.mapToObj(i -> "key" + String.format("%02d", i)).sorted().collect(Collectors.toList());
assertEquals(expectedKeys, hoodieReader.filterRowKeys(candidateRowKeys)

View File

@@ -18,6 +18,8 @@
package org.apache.hudi
import org.apache.hudi.common.config.TypedProperties
object HoodieConversionUtils {
def toJavaOption[T](opt: Option[T]): org.apache.hudi.common.util.Option[T] =
@@ -26,4 +28,10 @@ object HoodieConversionUtils {
def toScalaOption[T](opt: org.apache.hudi.common.util.Option[T]): Option[T] =
if (opt.isPresent) Some(opt.get) else None
def toProperties(params: Map[String, String]): TypedProperties = {
val props = new TypedProperties()
params.foreach(kv => props.setProperty(kv._1, kv._2))
props
}
}

View File

@@ -20,6 +20,7 @@ package org.apache.hudi.client.functional;
import org.apache.hudi.avro.HoodieAvroUtils;
import org.apache.hudi.avro.model.HoodieCleanMetadata;
import org.apache.hudi.avro.model.HoodieMetadataColumnStats;
import org.apache.hudi.avro.model.HoodieMetadataRecord;
import org.apache.hudi.client.SparkRDDWriteClient;
import org.apache.hudi.client.WriteStatus;
@@ -32,6 +33,7 @@ import org.apache.hudi.common.fs.ConsistencyGuardConfig;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.metrics.Registry;
import org.apache.hudi.common.model.FileSlice;
import org.apache.hudi.common.model.HoodieAvroRecord;
import org.apache.hudi.common.model.HoodieBaseFile;
import org.apache.hudi.common.model.HoodieCleaningPolicy;
import org.apache.hudi.common.model.HoodieCommitMetadata;
@@ -44,6 +46,7 @@ import org.apache.hudi.common.model.HoodieLogFile;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.model.HoodieWriteStat;
import org.apache.hudi.common.model.WriteConcurrencyMode;
import org.apache.hudi.common.table.HoodieTableConfig;
import org.apache.hudi.common.table.HoodieTableMetaClient;
@@ -69,6 +72,8 @@ import org.apache.hudi.common.util.HoodieTimer;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.StringUtils;
import org.apache.hudi.common.util.collection.ExternalSpillableMap;
import org.apache.hudi.common.util.hash.ColumnIndexID;
import org.apache.hudi.common.util.hash.PartitionIndexID;
import org.apache.hudi.config.HoodieClusteringConfig;
import org.apache.hudi.config.HoodieCompactionConfig;
import org.apache.hudi.config.HoodieIndexConfig;
@@ -84,6 +89,7 @@ import org.apache.hudi.metadata.HoodieMetadataMergedLogRecordReader;
import org.apache.hudi.metadata.HoodieMetadataMetrics;
import org.apache.hudi.metadata.HoodieMetadataPayload;
import org.apache.hudi.metadata.HoodieTableMetadata;
import org.apache.hudi.metadata.HoodieTableMetadataUtil;
import org.apache.hudi.metadata.MetadataPartitionType;
import org.apache.hudi.metadata.SparkHoodieBackedTableMetadataWriter;
import org.apache.hudi.table.HoodieSparkTable;
@@ -100,7 +106,6 @@ import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.util.Time;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
@@ -736,12 +741,12 @@ public class TestHoodieBackedMetadata extends TestHoodieMetadataBase {
HoodieBaseFile baseFile = fileSlices.get(0).getBaseFile().get();
HoodieHFileReader hoodieHFileReader = new HoodieHFileReader(context.getHadoopConf().get(), new Path(baseFile.getPath()),
new CacheConfig(context.getHadoopConf().get()));
List<Pair<String, IndexedRecord>> records = hoodieHFileReader.readAllRecords();
List<IndexedRecord> records = HoodieHFileReader.readAllRecords(hoodieHFileReader);
records.forEach(entry -> {
if (populateMetaFields) {
assertNotNull(((GenericRecord) entry.getSecond()).get(HoodieRecord.RECORD_KEY_METADATA_FIELD));
assertNotNull(((GenericRecord) entry).get(HoodieRecord.RECORD_KEY_METADATA_FIELD));
} else {
assertNull(((GenericRecord) entry.getSecond()).get(HoodieRecord.RECORD_KEY_METADATA_FIELD));
assertNull(((GenericRecord) entry).get(HoodieRecord.RECORD_KEY_METADATA_FIELD));
}
});
}
@@ -977,12 +982,11 @@ public class TestHoodieBackedMetadata extends TestHoodieMetadataBase {
}
Schema writerSchema = new AvroSchemaConverter().convert(writerSchemaMsg);
try (HoodieLogFormat.Reader logFileReader = HoodieLogFormat.newReader(fs, new HoodieLogFile(fsStatus[0].getPath()), writerSchema)) {
while (logFileReader.hasNext()) {
HoodieLogBlock logBlock = logFileReader.next();
if (logBlock instanceof HoodieDataBlock) {
try (ClosableIterator<IndexedRecord> recordItr = ((HoodieDataBlock) logBlock).getRecordItr()) {
try (ClosableIterator<IndexedRecord> recordItr = ((HoodieDataBlock) logBlock).getRecordIterator()) {
recordItr.forEachRemaining(indexRecord -> {
final GenericRecord record = (GenericRecord) indexRecord;
if (enableMetaFields) {
@@ -1068,15 +1072,15 @@ public class TestHoodieBackedMetadata extends TestHoodieMetadataBase {
HoodieHFileReader hoodieHFileReader = new HoodieHFileReader(context.getHadoopConf().get(),
new Path(baseFile.getPath()),
new CacheConfig(context.getHadoopConf().get()));
List<Pair<String, IndexedRecord>> records = hoodieHFileReader.readAllRecords();
List<IndexedRecord> records = HoodieHFileReader.readAllRecords(hoodieHFileReader);
records.forEach(entry -> {
if (enableMetaFields) {
assertNotNull(((GenericRecord) entry.getSecond()).get(HoodieRecord.RECORD_KEY_METADATA_FIELD));
assertNotNull(((GenericRecord) entry).get(HoodieRecord.RECORD_KEY_METADATA_FIELD));
} else {
assertNull(((GenericRecord) entry.getSecond()).get(HoodieRecord.RECORD_KEY_METADATA_FIELD));
assertNull(((GenericRecord) entry).get(HoodieRecord.RECORD_KEY_METADATA_FIELD));
}
final String keyInPayload = (String) ((GenericRecord) entry.getSecond())
final String keyInPayload = (String) ((GenericRecord) entry)
.get(HoodieMetadataPayload.KEY_FIELD_NAME);
assertFalse(keyInPayload.isEmpty());
});
@@ -1383,6 +1387,139 @@ public class TestHoodieBackedMetadata extends TestHoodieMetadataBase {
testTableOperationsImpl(engineContext, writeConfig);
}
@Test
public void testColStatsPrefixLookup() throws IOException {
this.tableType = COPY_ON_WRITE;
initPath();
initSparkContexts("TestHoodieMetadata");
initFileSystem();
fs.mkdirs(new Path(basePath));
initTimelineService();
initMetaClient(tableType);
initTestDataGenerator();
metadataTableBasePath = HoodieTableMetadata.getMetadataTableBasePath(basePath);
HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc);
// disable small file handling so that every insert goes to a new file group.
HoodieWriteConfig writeConfig = getWriteConfigBuilder(true, true, false)
.withRollbackUsingMarkers(false)
.withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(0)
.withInlineCompaction(false).withMaxNumDeltaCommitsBeforeCompaction(1)
.withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.EAGER)
.withAutoClean(false).retainCommits(1).retainFileVersions(1).build())
.withMetadataConfig(HoodieMetadataConfig.newBuilder()
.enable(true)
.withMetadataIndexColumnStats(true)
.enableFullScan(false)
.build())
.build();
try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, writeConfig)) {
String firstCommit = "0000001";
List<HoodieRecord> records = dataGen.generateInserts(firstCommit, 20);
AtomicInteger counter = new AtomicInteger();
List<HoodieRecord> processedRecords = records.stream().map(entry ->
new HoodieAvroRecord(new HoodieKey("key1_" + counter.getAndIncrement(), entry.getPartitionPath()), (HoodieRecordPayload) entry.getData()))
.collect(Collectors.toList());
client.startCommitWithTime(firstCommit);
List<WriteStatus> writeStatuses = client.insert(jsc.parallelize(processedRecords, 1), firstCommit).collect();
assertNoWriteErrors(writeStatuses);
// Write 2 (inserts)
String secondCommit = "0000002";
client.startCommitWithTime(secondCommit);
records = dataGen.generateInserts(secondCommit, 20);
AtomicInteger counter1 = new AtomicInteger();
processedRecords = records.stream().map(entry ->
new HoodieAvroRecord(new HoodieKey("key2_" + counter1.getAndIncrement(), entry.getPartitionPath()), (HoodieRecordPayload) entry.getData()))
.collect(Collectors.toList());
writeStatuses = client.insert(jsc.parallelize(processedRecords, 1), secondCommit).collect();
assertNoWriteErrors(writeStatuses);
Map<String, Map<String, List<String>>> commitToPartitionsToFiles = new HashMap<>();
// populate commit -> partition -> file info to assist in validation and prefi
metaClient.getActiveTimeline().getInstants().forEach(entry -> {
try {
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata
.fromBytes(metaClient.getActiveTimeline().getInstantDetails(entry).get(), HoodieCommitMetadata.class);
String commitTime = entry.getTimestamp();
if (!commitToPartitionsToFiles.containsKey(commitTime)) {
commitToPartitionsToFiles.put(commitTime, new HashMap<>());
}
commitMetadata.getPartitionToWriteStats().entrySet()
.stream()
.forEach(partitionWriteStat -> {
String partitionStatName = partitionWriteStat.getKey();
List<HoodieWriteStat> writeStats = partitionWriteStat.getValue();
String partition = HoodieTableMetadataUtil.getPartition(partitionStatName);
if (!commitToPartitionsToFiles.get(commitTime).containsKey(partition)) {
commitToPartitionsToFiles.get(commitTime).put(partition, new ArrayList<>());
}
writeStats.forEach(writeStat -> commitToPartitionsToFiles.get(commitTime).get(partition).add(writeStat.getPath()));
});
} catch (IOException e) {
e.printStackTrace();
}
});
HoodieTableMetadata tableMetadata = metadata(client);
// prefix search for column (_hoodie_record_key)
ColumnIndexID columnIndexID = new ColumnIndexID(HoodieRecord.RECORD_KEY_METADATA_FIELD);
List<HoodieRecord<HoodieMetadataPayload>> result = tableMetadata.getRecordsByKeyPrefixes(Collections.singletonList(columnIndexID.asBase64EncodedString()),
MetadataPartitionType.COLUMN_STATS.getPartitionPath()).collectAsList();
// there are 3 partitions in total and 2 commits. total entries should be 6.
assertEquals(result.size(), 6);
result.forEach(entry -> {
//LOG.warn("Prefix search entries just for record key col : " + entry.getRecordKey().toString() + " :: " + entry.getData().getColumnStatMetadata().get().toString());
});
// prefix search for col(_hoodie_record_key) and first partition. only 2 files should be matched
PartitionIndexID partitionIndexID = new PartitionIndexID(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH);
result = tableMetadata.getRecordsByKeyPrefixes(Collections.singletonList(columnIndexID.asBase64EncodedString().concat(partitionIndexID.asBase64EncodedString())),
MetadataPartitionType.COLUMN_STATS.getPartitionPath()).collectAsList();
// 1 partition and 2 commits. total entries should be 2.
assertEquals(result.size(), 2);
result.forEach(entry -> {
// LOG.warn("Prefix search entries for record key col and first partition : " + entry.getRecordKey().toString() + " :: " + entry.getData().getColumnStatMetadata().get().toString());
HoodieMetadataColumnStats metadataColumnStats = entry.getData().getColumnStatMetadata().get();
String fileName = metadataColumnStats.getFileName();
if (fileName.contains(firstCommit)) {
assertTrue(commitToPartitionsToFiles.get(firstCommit).get(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH)
.contains(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH + "/" + fileName));
} else {
assertTrue(commitToPartitionsToFiles.get(secondCommit).get(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH)
.contains(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH + "/" + fileName));
}
});
// prefix search for column {commit time} and first partition
columnIndexID = new ColumnIndexID(HoodieRecord.COMMIT_TIME_METADATA_FIELD);
result = tableMetadata.getRecordsByKeyPrefixes(Collections.singletonList(columnIndexID.asBase64EncodedString().concat(partitionIndexID.asBase64EncodedString())),
MetadataPartitionType.COLUMN_STATS.getPartitionPath()).collectAsList();
// 1 partition and 2 commits. total entries should be 2.
assertEquals(result.size(), 2);
result.forEach(entry -> {
// LOG.warn("Prefix search entries for record key col and first partition : " + entry.getRecordKey().toString() + " :: " + entry.getData().getColumnStatMetadata().get().toString());
HoodieMetadataColumnStats metadataColumnStats = entry.getData().getColumnStatMetadata().get();
// for commit time column, min max should be the same since we disable small files, every commit will create a new file
assertEquals(metadataColumnStats.getMinValue(), metadataColumnStats.getMaxValue());
String fileName = metadataColumnStats.getFileName();
if (fileName.contains(firstCommit)) {
assertTrue(commitToPartitionsToFiles.get(firstCommit).get(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH)
.contains(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH + "/" + fileName));
} else {
assertTrue(commitToPartitionsToFiles.get(secondCommit).get(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH)
.contains(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH + "/" + fileName));
}
});
}
}
/**
* Test all major table operations with the given table, config and context.
*
@@ -1476,8 +1613,8 @@ public class TestHoodieBackedMetadata extends TestHoodieMetadataBase {
Properties properties = new Properties();
properties.setProperty(FILESYSTEM_LOCK_PATH_PROP_KEY, basePath + "/.hoodie/.locks");
properties.setProperty(LockConfiguration.LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY,"1000");
properties.setProperty(LockConfiguration.LOCK_ACQUIRE_CLIENT_NUM_RETRIES_PROP_KEY,"20");
properties.setProperty(LockConfiguration.LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY, "1000");
properties.setProperty(LockConfiguration.LOCK_ACQUIRE_CLIENT_NUM_RETRIES_PROP_KEY, "20");
HoodieWriteConfig writeConfig = getWriteConfigBuilder(true, true, false)
.withCompactionConfig(HoodieCompactionConfig.newBuilder()
.withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY).withAutoClean(false).build())
@@ -1540,7 +1677,7 @@ public class TestHoodieBackedMetadata extends TestHoodieMetadataBase {
Properties properties = new Properties();
properties.setProperty(FILESYSTEM_LOCK_PATH_PROP_KEY, basePath + "/.hoodie/.locks");
properties.setProperty(LockConfiguration.LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY,"3000");
properties.setProperty(LockConfiguration.LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY, "3000");
HoodieWriteConfig writeConfig = getWriteConfigBuilder(true, true, false)
.withCompactionConfig(HoodieCompactionConfig.newBuilder()
@@ -1871,7 +2008,7 @@ public class TestHoodieBackedMetadata extends TestHoodieMetadataBase {
Properties properties = new Properties();
properties.setProperty(FILESYSTEM_LOCK_PATH_PROP_KEY, basePath + "/.hoodie/.locks");
properties.setProperty(LockConfiguration.LOCK_ACQUIRE_CLIENT_NUM_RETRIES_PROP_KEY, "3");
properties.setProperty(LockConfiguration.LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY,"3000");
properties.setProperty(LockConfiguration.LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY, "3000");
HoodieWriteConfig writeConfig = getWriteConfigBuilder(false, true, false)
.withCompactionConfig(HoodieCompactionConfig.newBuilder()
.withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY).withAutoClean(false).build())
@@ -2364,7 +2501,7 @@ public class TestHoodieBackedMetadata extends TestHoodieMetadataBase {
while (logFileReader.hasNext()) {
HoodieLogBlock logBlock = logFileReader.next();
if (logBlock instanceof HoodieDataBlock) {
try (ClosableIterator<IndexedRecord> recordItr = ((HoodieDataBlock) logBlock).getRecordItr()) {
try (ClosableIterator<IndexedRecord> recordItr = ((HoodieDataBlock) logBlock).getRecordIterator()) {
recordItr.forEachRemaining(indexRecord -> {
final GenericRecord record = (GenericRecord) indexRecord;
final GenericRecord colStatsRecord = (GenericRecord) record.get(HoodieMetadataPayload.SCHEMA_FIELD_ID_COLUMN_STATS);

View File

@@ -21,9 +21,9 @@ package org.apache.hudi.client.functional;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hudi.avro.HoodieAvroUtils;
import org.apache.hudi.avro.model.HoodieMetadataRecord;
import org.apache.hudi.common.config.HoodieMetadataConfig;
@@ -51,8 +51,6 @@ import org.apache.hudi.metadata.HoodieTableMetadataKeyGenerator;
import org.apache.hudi.metadata.MetadataPartitionType;
import org.apache.hudi.table.HoodieSparkTable;
import org.apache.hudi.table.HoodieTable;
import org.apache.hadoop.fs.FileStatus;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.parquet.avro.AvroSchemaConverter;
@@ -292,7 +290,7 @@ public class TestHoodieBackedTableMetadata extends TestHoodieMetadataBase {
while (logFileReader.hasNext()) {
HoodieLogBlock logBlock = logFileReader.next();
if (logBlock instanceof HoodieDataBlock) {
try (ClosableIterator<IndexedRecord> recordItr = ((HoodieDataBlock) logBlock).getRecordItr()) {
try (ClosableIterator<IndexedRecord> recordItr = ((HoodieDataBlock) logBlock).getRecordIterator()) {
recordItr.forEachRemaining(indexRecord -> {
final GenericRecord record = (GenericRecord) indexRecord;
assertNull(record.get(HoodieRecord.RECORD_KEY_METADATA_FIELD));
@@ -361,10 +359,10 @@ public class TestHoodieBackedTableMetadata extends TestHoodieMetadataBase {
HoodieHFileReader hoodieHFileReader = new HoodieHFileReader(context.getHadoopConf().get(),
new Path(baseFile.getPath()),
new CacheConfig(context.getHadoopConf().get()));
List<Pair<String, IndexedRecord>> records = hoodieHFileReader.readAllRecords();
List<IndexedRecord> records = HoodieHFileReader.readAllRecords(hoodieHFileReader);
records.forEach(entry -> {
assertNull(((GenericRecord) entry.getSecond()).get(HoodieRecord.RECORD_KEY_METADATA_FIELD));
final String keyInPayload = (String) ((GenericRecord) entry.getSecond())
assertNull(((GenericRecord) entry).get(HoodieRecord.RECORD_KEY_METADATA_FIELD));
final String keyInPayload = (String) ((GenericRecord) entry)
.get(HoodieMetadataPayload.KEY_FIELD_NAME);
assertFalse(keyInPayload.isEmpty());
});

View File

@@ -67,7 +67,7 @@ import java.util.List;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import static org.apache.hudi.io.storage.HoodieHFileReader.KEY_SCHEMA;
import static org.apache.hudi.io.storage.HoodieHFileReader.SCHEMA_KEY;
/**
* Utility methods to aid testing inside the HoodieClient module.
@@ -247,7 +247,7 @@ public class HoodieClientTestUtils {
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())));
schema = new Schema.Parser().parse(new String(reader.getHFileInfo().get(SCHEMA_KEY.getBytes())));
}
HFileScanner scanner = reader.getScanner(false, false);
if (!scanner.seekTo()) {