|
|
|
|
@@ -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);
|
|
|
|
|
|