[HUDI-2488][HUDI-3175] Implement async metadata indexing (#4693)
- Add a new action called INDEX, whose state transition is described in the RFC. - Changes in timeline to support the new action. - Add an index planner in ScheduleIndexActionExecutor. - Add index plan executor in RunIndexActionExecutor. - Add 3 APIs in HoodieTableMetadataWriter; a) scheduleIndex: will generate an index plan based on latest completed instant, initialize file groups and add a requested INDEX instant, b) index: executes the index plan and also takes care of writes that happened after indexing was requested, c) dropIndex: will drop index by removing the given metadata partition. - Add 2 new table configs to serve as the source of truth for inflight and completed indexes. - Support upgrade/downgrade taking care of the newly added configs. - Add tool to trigger indexing in HoodieIndexer. - Handle corner cases related to partial failures. - Abort gracefully after deleting partition and instant. - Handle other actions in timeline to consider before catching up
This commit is contained in:
@@ -120,6 +120,14 @@
|
||||
"HoodieCommitMetadata"
|
||||
],
|
||||
"default": null
|
||||
},
|
||||
{
|
||||
"name":"hoodieIndexCommitMetadata",
|
||||
"type":[
|
||||
"null",
|
||||
"HoodieIndexCommitMetadata"
|
||||
],
|
||||
"default": null
|
||||
}
|
||||
]
|
||||
}
|
||||
|
||||
54
hudi-common/src/main/avro/HoodieIndexCommitMetadata.avsc
Normal file
54
hudi-common/src/main/avro/HoodieIndexCommitMetadata.avsc
Normal file
@@ -0,0 +1,54 @@
|
||||
/*
|
||||
* 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.
|
||||
*/
|
||||
{
|
||||
"namespace": "org.apache.hudi.avro.model",
|
||||
"type": "record",
|
||||
"name": "HoodieIndexCommitMetadata",
|
||||
"fields": [
|
||||
{
|
||||
"name": "version",
|
||||
"doc": "This field replaces the field filesToBeDeletedPerPartition",
|
||||
"type": [
|
||||
"int",
|
||||
"null"
|
||||
],
|
||||
"default": 1
|
||||
},
|
||||
{
|
||||
"name": "operationType",
|
||||
"doc": "This field replaces the field filesToBeDeletedPerPartition",
|
||||
"type": [
|
||||
"null",
|
||||
"string"
|
||||
],
|
||||
"default": null
|
||||
},
|
||||
{
|
||||
"name": "indexPartitionInfos",
|
||||
"doc": "This field contains the info for each partition that got indexed",
|
||||
"type": [
|
||||
"null",
|
||||
{
|
||||
"type": "array",
|
||||
"items": "HoodieIndexPartitionInfo"
|
||||
}
|
||||
],
|
||||
"default": null
|
||||
}
|
||||
]
|
||||
}
|
||||
48
hudi-common/src/main/avro/HoodieIndexPartitionInfo.avsc
Normal file
48
hudi-common/src/main/avro/HoodieIndexPartitionInfo.avsc
Normal file
@@ -0,0 +1,48 @@
|
||||
/*
|
||||
* 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.
|
||||
*/
|
||||
{
|
||||
"namespace": "org.apache.hudi.avro.model",
|
||||
"type": "record",
|
||||
"name": "HoodieIndexPartitionInfo",
|
||||
"fields": [
|
||||
{
|
||||
"name": "version",
|
||||
"type": [
|
||||
"int",
|
||||
"null"
|
||||
],
|
||||
"default": 1
|
||||
},
|
||||
{
|
||||
"name": "metadataPartitionPath",
|
||||
"type": [
|
||||
"null",
|
||||
"string"
|
||||
],
|
||||
"default": null
|
||||
},
|
||||
{
|
||||
"name": "indexUptoInstant",
|
||||
"type": [
|
||||
"null",
|
||||
"string"
|
||||
],
|
||||
"default": null
|
||||
}
|
||||
]
|
||||
}
|
||||
43
hudi-common/src/main/avro/HoodieIndexPlan.avsc
Normal file
43
hudi-common/src/main/avro/HoodieIndexPlan.avsc
Normal file
@@ -0,0 +1,43 @@
|
||||
/*
|
||||
* 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.
|
||||
*/
|
||||
{
|
||||
"namespace": "org.apache.hudi.avro.model",
|
||||
"type": "record",
|
||||
"name": "HoodieIndexPlan",
|
||||
"fields": [
|
||||
{
|
||||
"name": "version",
|
||||
"type": [
|
||||
"int",
|
||||
"null"
|
||||
],
|
||||
"default": 1
|
||||
},
|
||||
{
|
||||
"name": "indexPartitionInfos",
|
||||
"type": [
|
||||
"null",
|
||||
{
|
||||
"type": "array",
|
||||
"items": "HoodieIndexPartitionInfo"
|
||||
}
|
||||
],
|
||||
"default": null
|
||||
}
|
||||
]
|
||||
}
|
||||
@@ -78,7 +78,7 @@ public class SimpleBloomFilter implements BloomFilter {
|
||||
@Override
|
||||
public void add(String key) {
|
||||
if (key == null) {
|
||||
throw new NullPointerException("Key cannot by null");
|
||||
throw new NullPointerException("Key cannot be null");
|
||||
}
|
||||
filter.add(new Key(key.getBytes(StandardCharsets.UTF_8)));
|
||||
}
|
||||
@@ -86,7 +86,7 @@ public class SimpleBloomFilter implements BloomFilter {
|
||||
@Override
|
||||
public boolean mightContain(String key) {
|
||||
if (key == null) {
|
||||
throw new NullPointerException("Key cannot by null");
|
||||
throw new NullPointerException("Key cannot be null");
|
||||
}
|
||||
return filter.membershipTest(new Key(key.getBytes(StandardCharsets.UTF_8)));
|
||||
}
|
||||
|
||||
@@ -71,6 +71,13 @@ public final class HoodieMetadataConfig extends HoodieConfig {
|
||||
.sinceVersion("0.7.0")
|
||||
.withDocumentation("Enable asynchronous cleaning for metadata table");
|
||||
|
||||
// Async index
|
||||
public static final ConfigProperty<Boolean> ASYNC_INDEX_ENABLE = ConfigProperty
|
||||
.key(METADATA_PREFIX + ".index.async")
|
||||
.defaultValue(false)
|
||||
.sinceVersion("0.11.0")
|
||||
.withDocumentation("Enable asynchronous indexing of metadata table.");
|
||||
|
||||
// Maximum delta commits before compaction occurs
|
||||
public static final ConfigProperty<Integer> COMPACT_NUM_DELTA_COMMITS = ConfigProperty
|
||||
.key(METADATA_PREFIX + ".compact.max.delta.commits")
|
||||
@@ -175,6 +182,25 @@ public final class HoodieMetadataConfig extends HoodieConfig {
|
||||
.sinceVersion("0.11.0")
|
||||
.withDocumentation("Parallelism to use, when generating column stats index.");
|
||||
|
||||
public static final ConfigProperty<String> COLUMN_STATS_INDEX_FOR_COLUMNS = ConfigProperty
|
||||
.key(METADATA_PREFIX + ".index.column.stats.column.list")
|
||||
.noDefaultValue()
|
||||
.sinceVersion("0.11.0")
|
||||
.withDocumentation("Comma-separated list of columns for which column stats index will be built. If not set, all columns will be indexed");
|
||||
|
||||
public static final ConfigProperty<String> BLOOM_FILTER_INDEX_FOR_COLUMNS = ConfigProperty
|
||||
.key(METADATA_PREFIX + ".index.bloom.filter.column.list")
|
||||
.noDefaultValue()
|
||||
.sinceVersion("0.11.0")
|
||||
.withDocumentation("Comma-separated list of columns for which bloom filter index will be built. If not set, only record key will be indexed.");
|
||||
|
||||
public static final ConfigProperty<Integer> METADATA_INDEX_CHECK_TIMEOUT_SECONDS = ConfigProperty
|
||||
.key(METADATA_PREFIX + ".index.check.timeout.seconds")
|
||||
.defaultValue(900)
|
||||
.sinceVersion("0.11.0")
|
||||
.withDocumentation("After the async indexer has finished indexing upto the base instant, it will ensure that all inflight writers "
|
||||
+ "reliably write index updates as well. If this timeout expires, then the indexer will abort itself safely.");
|
||||
|
||||
public static final ConfigProperty<Boolean> POPULATE_META_FIELDS = ConfigProperty
|
||||
.key(METADATA_PREFIX + ".populate.meta.fields")
|
||||
.defaultValue(false)
|
||||
@@ -221,6 +247,14 @@ public final class HoodieMetadataConfig extends HoodieConfig {
|
||||
return getBooleanOrDefault(ENABLE_METADATA_INDEX_COLUMN_STATS_FOR_ALL_COLUMNS);
|
||||
}
|
||||
|
||||
public String getColumnsEnabledForColumnStatsIndex() {
|
||||
return getString(COLUMN_STATS_INDEX_FOR_COLUMNS);
|
||||
}
|
||||
|
||||
public String getColumnsEnabledForBloomFilterIndex() {
|
||||
return getString(BLOOM_FILTER_INDEX_FOR_COLUMNS);
|
||||
}
|
||||
|
||||
public int getBloomFilterIndexFileGroupCount() {
|
||||
return getIntOrDefault(METADATA_INDEX_BLOOM_FILTER_FILE_GROUP_COUNT);
|
||||
}
|
||||
@@ -233,6 +267,10 @@ public final class HoodieMetadataConfig extends HoodieConfig {
|
||||
return getIntOrDefault(COLUMN_STATS_INDEX_PARALLELISM);
|
||||
}
|
||||
|
||||
public int getIndexingCheckTimeoutSeconds() {
|
||||
return getIntOrDefault(METADATA_INDEX_CHECK_TIMEOUT_SECONDS);
|
||||
}
|
||||
|
||||
public boolean enableMetrics() {
|
||||
return getBoolean(METRICS_ENABLE);
|
||||
}
|
||||
@@ -305,6 +343,21 @@ public final class HoodieMetadataConfig extends HoodieConfig {
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withColumnStatsIndexForColumns(String columns) {
|
||||
metadataConfig.setValue(COLUMN_STATS_INDEX_FOR_COLUMNS, columns);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withBloomFilterIndexForColumns(String columns) {
|
||||
metadataConfig.setValue(BLOOM_FILTER_INDEX_FOR_COLUMNS, columns);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withIndexingCheckTimeout(int timeoutInSeconds) {
|
||||
metadataConfig.setValue(METADATA_INDEX_CHECK_TIMEOUT_SECONDS, String.valueOf(timeoutInSeconds));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder enableMetrics(boolean enableMetrics) {
|
||||
metadataConfig.setValue(METRICS_ENABLE, String.valueOf(enableMetrics));
|
||||
return this;
|
||||
@@ -320,6 +373,11 @@ public final class HoodieMetadataConfig extends HoodieConfig {
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withAsyncIndex(boolean asyncIndex) {
|
||||
metadataConfig.setValue(ASYNC_INDEX_ENABLE, String.valueOf(asyncIndex));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withMaxNumDeltaCommitsBeforeCompaction(int maxNumDeltaCommitsBeforeCompaction) {
|
||||
metadataConfig.setValue(COMPACT_NUM_DELTA_COMMITS, String.valueOf(maxNumDeltaCommitsBeforeCompaction));
|
||||
return this;
|
||||
|
||||
@@ -48,6 +48,8 @@ public enum WriteOperationType {
|
||||
INSERT_OVERWRITE_TABLE("insert_overwrite_table"),
|
||||
// compact
|
||||
COMPACT("compact"),
|
||||
|
||||
INDEX("index"),
|
||||
// used for old version
|
||||
UNKNOWN("unknown");
|
||||
|
||||
@@ -86,6 +88,8 @@ public enum WriteOperationType {
|
||||
return CLUSTER;
|
||||
case "compact":
|
||||
return COMPACT;
|
||||
case "index":
|
||||
return INDEX;
|
||||
case "unknown":
|
||||
return UNKNOWN;
|
||||
default:
|
||||
|
||||
@@ -36,6 +36,7 @@ import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion;
|
||||
import org.apache.hudi.common.util.BinaryUtil;
|
||||
import org.apache.hudi.common.util.FileIOUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.StringUtils;
|
||||
import org.apache.hudi.common.util.ValidationUtils;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.keygen.constant.KeyGeneratorOptions;
|
||||
@@ -208,6 +209,20 @@ public class HoodieTableConfig extends HoodieConfig {
|
||||
.sinceVersion("0.11.0")
|
||||
.withDocumentation("Table checksum is used to guard against partial writes in HDFS. It is added as the last entry in hoodie.properties and then used to validate while reading table config.");
|
||||
|
||||
public static final ConfigProperty<String> TABLE_METADATA_PARTITIONS_INFLIGHT = ConfigProperty
|
||||
.key("hoodie.table.metadata.partitions.inflight")
|
||||
.noDefaultValue()
|
||||
.sinceVersion("0.11.0")
|
||||
.withDocumentation("Comma-separated list of metadata partitions whose building is in progress. "
|
||||
+ "These partitions are not yet ready for use by the readers.");
|
||||
|
||||
public static final ConfigProperty<String> TABLE_METADATA_PARTITIONS = ConfigProperty
|
||||
.key("hoodie.table.metadata.partitions")
|
||||
.noDefaultValue()
|
||||
.sinceVersion("0.11.0")
|
||||
.withDocumentation("Comma-separated list of metadata partitions that have been completely built and in-sync with data table. "
|
||||
+ "These partitions are ready for use by the readers");
|
||||
|
||||
private static final String TABLE_CHECKSUM_FORMAT = "%s.%s"; // <database_name>.<table_name>
|
||||
|
||||
public HoodieTableConfig(FileSystem fs, String metaPath, String payloadClassName) {
|
||||
@@ -585,6 +600,14 @@ public class HoodieTableConfig extends HoodieConfig {
|
||||
return getLong(TABLE_CHECKSUM);
|
||||
}
|
||||
|
||||
public String getMetadataPartitionsInflight() {
|
||||
return getStringOrDefault(TABLE_METADATA_PARTITIONS_INFLIGHT, StringUtils.EMPTY_STRING);
|
||||
}
|
||||
|
||||
public String getMetadataPartitions() {
|
||||
return getStringOrDefault(TABLE_METADATA_PARTITIONS, StringUtils.EMPTY_STRING);
|
||||
}
|
||||
|
||||
public Map<String, String> propsMap() {
|
||||
return props.entrySet().stream()
|
||||
.collect(Collectors.toMap(e -> String.valueOf(e.getKey()), e -> String.valueOf(e.getValue())));
|
||||
|
||||
@@ -73,7 +73,8 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
|
||||
INFLIGHT_COMPACTION_EXTENSION, REQUESTED_COMPACTION_EXTENSION,
|
||||
REQUESTED_RESTORE_EXTENSION, INFLIGHT_RESTORE_EXTENSION, RESTORE_EXTENSION,
|
||||
ROLLBACK_EXTENSION, REQUESTED_ROLLBACK_EXTENSION, INFLIGHT_ROLLBACK_EXTENSION,
|
||||
REQUESTED_REPLACE_COMMIT_EXTENSION, INFLIGHT_REPLACE_COMMIT_EXTENSION, REPLACE_COMMIT_EXTENSION));
|
||||
REQUESTED_REPLACE_COMMIT_EXTENSION, INFLIGHT_REPLACE_COMMIT_EXTENSION, REPLACE_COMMIT_EXTENSION,
|
||||
REQUESTED_INDEX_COMMIT_EXTENSION, INFLIGHT_INDEX_COMMIT_EXTENSION, INDEX_COMMIT_EXTENSION));
|
||||
private static final Logger LOG = LogManager.getLogger(HoodieActiveTimeline.class);
|
||||
protected HoodieTableMetaClient metaClient;
|
||||
|
||||
@@ -99,7 +100,6 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
|
||||
return HoodieInstantTimeGenerator.createNewInstantTime(0);
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Returns next instant time that adds N milliseconds to current time.
|
||||
* Ensures each instant time is atleast 1 second apart since we create instant times at second granularity
|
||||
@@ -225,7 +225,7 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
|
||||
deleteInstantFile(instant);
|
||||
}
|
||||
|
||||
private void deleteInstantFileIfExists(HoodieInstant instant) {
|
||||
public void deleteInstantFileIfExists(HoodieInstant instant) {
|
||||
LOG.info("Deleting instant " + instant);
|
||||
Path inFlightCommitFilePath = new Path(metaClient.getMetaPath(), instant.getFileName());
|
||||
try {
|
||||
@@ -339,6 +339,10 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
|
||||
}
|
||||
}
|
||||
|
||||
public Option<byte[]> readIndexPlanAsBytes(HoodieInstant instant) {
|
||||
return readDataFromPath(new Path(metaClient.getMetaPath(), instant.getFileName()));
|
||||
}
|
||||
|
||||
/**
|
||||
* Revert compaction State from inflight to requested.
|
||||
*
|
||||
@@ -652,6 +656,65 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
|
||||
createFileInMetaPath(instant.getFileName(), content, false);
|
||||
}
|
||||
|
||||
/**
|
||||
* Transition index instant state from requested to inflight.
|
||||
*
|
||||
* @param requestedInstant Inflight Instant
|
||||
* @return inflight instant
|
||||
*/
|
||||
public HoodieInstant transitionIndexRequestedToInflight(HoodieInstant requestedInstant, Option<byte[]> data) {
|
||||
ValidationUtils.checkArgument(requestedInstant.getAction().equals(HoodieTimeline.INDEXING_ACTION),
|
||||
String.format("%s is not equal to %s action", requestedInstant.getAction(), INDEXING_ACTION));
|
||||
ValidationUtils.checkArgument(requestedInstant.isRequested(),
|
||||
String.format("Instant %s not in requested state", requestedInstant.getTimestamp()));
|
||||
HoodieInstant inflightInstant = new HoodieInstant(State.INFLIGHT, INDEXING_ACTION, requestedInstant.getTimestamp());
|
||||
transitionState(requestedInstant, inflightInstant, data);
|
||||
return inflightInstant;
|
||||
}
|
||||
|
||||
/**
|
||||
* Transition index instant state from inflight to completed.
|
||||
* @param inflightInstant Inflight Instant
|
||||
* @return completed instant
|
||||
*/
|
||||
public HoodieInstant transitionIndexInflightToComplete(HoodieInstant inflightInstant, Option<byte[]> data) {
|
||||
ValidationUtils.checkArgument(inflightInstant.getAction().equals(HoodieTimeline.INDEXING_ACTION),
|
||||
String.format("%s is not equal to %s action", inflightInstant.getAction(), INDEXING_ACTION));
|
||||
ValidationUtils.checkArgument(inflightInstant.isInflight(),
|
||||
String.format("Instant %s not inflight", inflightInstant.getTimestamp()));
|
||||
HoodieInstant commitInstant = new HoodieInstant(State.COMPLETED, INDEXING_ACTION, inflightInstant.getTimestamp());
|
||||
transitionState(inflightInstant, commitInstant, data);
|
||||
return commitInstant;
|
||||
}
|
||||
|
||||
/**
|
||||
* Revert index instant state from inflight to requested.
|
||||
* @param inflightInstant Inflight Instant
|
||||
* @return requested instant
|
||||
*/
|
||||
public HoodieInstant revertIndexInflightToRequested(HoodieInstant inflightInstant) {
|
||||
ValidationUtils.checkArgument(inflightInstant.getAction().equals(HoodieTimeline.INDEXING_ACTION),
|
||||
String.format("%s is not equal to %s action", inflightInstant.getAction(), INDEXING_ACTION));
|
||||
ValidationUtils.checkArgument(inflightInstant.isInflight(),
|
||||
String.format("Instant %s not inflight", inflightInstant.getTimestamp()));
|
||||
HoodieInstant requestedInstant = new HoodieInstant(State.REQUESTED, INDEXING_ACTION, inflightInstant.getTimestamp());
|
||||
if (metaClient.getTimelineLayoutVersion().isNullVersion()) {
|
||||
transitionState(inflightInstant, requestedInstant, Option.empty());
|
||||
} else {
|
||||
deleteInflight(inflightInstant);
|
||||
}
|
||||
return requestedInstant;
|
||||
}
|
||||
|
||||
/**
|
||||
* Save content for inflight/requested index instant.
|
||||
*/
|
||||
public void saveToPendingIndexAction(HoodieInstant instant, Option<byte[]> content) {
|
||||
ValidationUtils.checkArgument(instant.getAction().equals(HoodieTimeline.INDEXING_ACTION),
|
||||
String.format("%s is not equal to %s action", instant.getAction(), INDEXING_ACTION));
|
||||
createFileInMetaPath(instant.getFileName(), content, false);
|
||||
}
|
||||
|
||||
private void createFileInMetaPath(String filename, Option<byte[]> content, boolean allowOverwrite) {
|
||||
Path fullPath = new Path(metaClient.getMetaPath(), filename);
|
||||
if (allowOverwrite || metaClient.getTimelineLayoutVersion().isNullVersion()) {
|
||||
|
||||
@@ -118,7 +118,8 @@ public class HoodieArchivedTimeline extends HoodieDefaultTimeline {
|
||||
*
|
||||
* @deprecated
|
||||
*/
|
||||
public HoodieArchivedTimeline() {}
|
||||
public HoodieArchivedTimeline() {
|
||||
}
|
||||
|
||||
/**
|
||||
* This method is only used when this object is deserialized in a spark executor.
|
||||
@@ -207,6 +208,8 @@ public class HoodieArchivedTimeline extends HoodieDefaultTimeline {
|
||||
return Option.of("hoodieCompactionPlan");
|
||||
case HoodieTimeline.REPLACE_COMMIT_ACTION:
|
||||
return Option.of("hoodieReplaceCommitMetadata");
|
||||
case HoodieTimeline.INDEXING_ACTION:
|
||||
return Option.of("hoodieIndexCommitMetadata");
|
||||
default:
|
||||
LOG.error(String.format("Unknown action in metadata (%s)", action));
|
||||
return Option.empty();
|
||||
|
||||
@@ -75,7 +75,8 @@ public class HoodieDefaultTimeline implements HoodieTimeline {
|
||||
*
|
||||
* @deprecated
|
||||
*/
|
||||
public HoodieDefaultTimeline() {}
|
||||
public HoodieDefaultTimeline() {
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieTimeline filterInflights() {
|
||||
@@ -112,6 +113,16 @@ public class HoodieDefaultTimeline implements HoodieTimeline {
|
||||
return new HoodieDefaultTimeline(instants.stream().filter(s -> validActions.contains(s.getAction())), details);
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieTimeline getContiguousCompletedWriteTimeline() {
|
||||
Option<HoodieInstant> earliestPending = getWriteTimeline().filterInflightsAndRequested().firstInstant();
|
||||
if (earliestPending.isPresent()) {
|
||||
return getWriteTimeline().filterCompletedInstants()
|
||||
.filter(instant -> HoodieTimeline.compareTimestamps(instant.getTimestamp(), LESSER_THAN, earliestPending.get().getTimestamp()));
|
||||
}
|
||||
return getWriteTimeline().filterCompletedInstants();
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieTimeline getCompletedReplaceTimeline() {
|
||||
return new HoodieDefaultTimeline(
|
||||
@@ -181,6 +192,16 @@ public class HoodieDefaultTimeline implements HoodieTimeline {
|
||||
return new HoodieDefaultTimeline(instants.stream().filter(filter), details);
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieTimeline filterPendingIndexTimeline() {
|
||||
return new HoodieDefaultTimeline(instants.stream().filter(s -> s.getAction().equals(INDEXING_ACTION) && !s.isCompleted()), details);
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieTimeline filterCompletedIndexTimeline() {
|
||||
return new HoodieDefaultTimeline(instants.stream().filter(s -> s.getAction().equals(INDEXING_ACTION) && s.isCompleted()), details);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get all instants (commits, delta commits) that produce new data, in the active timeline.
|
||||
*/
|
||||
@@ -189,12 +210,12 @@ public class HoodieDefaultTimeline implements HoodieTimeline {
|
||||
}
|
||||
|
||||
/**
|
||||
* Get all instants (commits, delta commits, compaction, clean, savepoint, rollback) that result in actions,
|
||||
* Get all instants (commits, delta commits, compaction, clean, savepoint, rollback, replace commits, index) that result in actions,
|
||||
* in the active timeline.
|
||||
*/
|
||||
public HoodieTimeline getAllCommitsTimeline() {
|
||||
return getTimelineOfActions(CollectionUtils.createSet(COMMIT_ACTION, DELTA_COMMIT_ACTION,
|
||||
CLEAN_ACTION, COMPACTION_ACTION, SAVEPOINT_ACTION, ROLLBACK_ACTION, REPLACE_COMMIT_ACTION));
|
||||
CLEAN_ACTION, COMPACTION_ACTION, SAVEPOINT_ACTION, ROLLBACK_ACTION, REPLACE_COMMIT_ACTION, INDEXING_ACTION));
|
||||
}
|
||||
|
||||
/**
|
||||
|
||||
@@ -172,6 +172,10 @@ public class HoodieInstant implements Serializable, Comparable<HoodieInstant> {
|
||||
return isInflight() ? HoodieTimeline.makeInflightReplaceFileName(timestamp)
|
||||
: isRequested() ? HoodieTimeline.makeRequestedReplaceFileName(timestamp)
|
||||
: HoodieTimeline.makeReplaceFileName(timestamp);
|
||||
} else if (HoodieTimeline.INDEXING_ACTION.equals(action)) {
|
||||
return isInflight() ? HoodieTimeline.makeInflightIndexFileName(timestamp)
|
||||
: isRequested() ? HoodieTimeline.makeRequestedIndexFileName(timestamp)
|
||||
: HoodieTimeline.makeIndexCommitFileName(timestamp);
|
||||
}
|
||||
throw new IllegalArgumentException("Cannot get file name for unknown action " + action);
|
||||
}
|
||||
|
||||
@@ -55,10 +55,11 @@ public interface HoodieTimeline extends Serializable {
|
||||
String COMPACTION_ACTION = "compaction";
|
||||
String REQUESTED_EXTENSION = ".requested";
|
||||
String RESTORE_ACTION = "restore";
|
||||
String INDEXING_ACTION = "indexing";
|
||||
|
||||
String[] VALID_ACTIONS_IN_TIMELINE = {COMMIT_ACTION, DELTA_COMMIT_ACTION,
|
||||
CLEAN_ACTION, SAVEPOINT_ACTION, RESTORE_ACTION, ROLLBACK_ACTION,
|
||||
COMPACTION_ACTION, REPLACE_COMMIT_ACTION};
|
||||
COMPACTION_ACTION, REPLACE_COMMIT_ACTION, INDEXING_ACTION};
|
||||
|
||||
String COMMIT_EXTENSION = "." + COMMIT_ACTION;
|
||||
String DELTA_COMMIT_EXTENSION = "." + DELTA_COMMIT_ACTION;
|
||||
@@ -84,6 +85,9 @@ public interface HoodieTimeline extends Serializable {
|
||||
String INFLIGHT_REPLACE_COMMIT_EXTENSION = "." + REPLACE_COMMIT_ACTION + INFLIGHT_EXTENSION;
|
||||
String REQUESTED_REPLACE_COMMIT_EXTENSION = "." + REPLACE_COMMIT_ACTION + REQUESTED_EXTENSION;
|
||||
String REPLACE_COMMIT_EXTENSION = "." + REPLACE_COMMIT_ACTION;
|
||||
String INFLIGHT_INDEX_COMMIT_EXTENSION = "." + INDEXING_ACTION + INFLIGHT_EXTENSION;
|
||||
String REQUESTED_INDEX_COMMIT_EXTENSION = "." + INDEXING_ACTION + REQUESTED_EXTENSION;
|
||||
String INDEX_COMMIT_EXTENSION = "." + INDEXING_ACTION;
|
||||
|
||||
String INVALID_INSTANT_TS = "0";
|
||||
|
||||
@@ -139,6 +143,15 @@ public interface HoodieTimeline extends Serializable {
|
||||
*/
|
||||
HoodieTimeline getWriteTimeline();
|
||||
|
||||
/**
|
||||
* Timeline to just include commits (commit/deltacommit), compaction and replace actions that are completed and contiguous.
|
||||
* For example, if timeline is [C0.completed, C1.completed, C2.completed, C3.inflight, C4.completed].
|
||||
* Then, a timeline of [C0.completed, C1.completed, C2.completed] will be returned.
|
||||
*
|
||||
* @return
|
||||
*/
|
||||
HoodieTimeline getContiguousCompletedWriteTimeline();
|
||||
|
||||
/**
|
||||
* Timeline to just include replace instants that have valid (commit/deltacommit) actions.
|
||||
*
|
||||
@@ -198,6 +211,16 @@ public interface HoodieTimeline extends Serializable {
|
||||
*/
|
||||
HoodieTimeline filter(Predicate<HoodieInstant> filter);
|
||||
|
||||
/**
|
||||
* Filter this timeline to just include requested and inflight index instants.
|
||||
*/
|
||||
HoodieTimeline filterPendingIndexTimeline();
|
||||
|
||||
/**
|
||||
* Filter this timeline to just include completed index instants.
|
||||
*/
|
||||
HoodieTimeline filterCompletedIndexTimeline();
|
||||
|
||||
/**
|
||||
* If the timeline has any instants.
|
||||
*
|
||||
@@ -341,6 +364,14 @@ public interface HoodieTimeline extends Serializable {
|
||||
return instant.isRequested() ? instant : HoodieTimeline.getRequestedInstant(instant);
|
||||
}
|
||||
|
||||
static HoodieInstant getIndexRequestedInstant(final String timestamp) {
|
||||
return new HoodieInstant(State.REQUESTED, INDEXING_ACTION, timestamp);
|
||||
}
|
||||
|
||||
static HoodieInstant getIndexInflightInstant(final String timestamp) {
|
||||
return new HoodieInstant(State.INFLIGHT, INDEXING_ACTION, timestamp);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the inflight instant corresponding to the instant being passed. Takes care of changes in action names
|
||||
* between inflight and completed instants (compaction <=> commit).
|
||||
@@ -454,4 +485,16 @@ public interface HoodieTimeline extends Serializable {
|
||||
static String makeFileNameAsInflight(String fileName) {
|
||||
return StringUtils.join(fileName, HoodieTimeline.INFLIGHT_EXTENSION);
|
||||
}
|
||||
|
||||
static String makeIndexCommitFileName(String instant) {
|
||||
return StringUtils.join(instant, HoodieTimeline.INDEX_COMMIT_EXTENSION);
|
||||
}
|
||||
|
||||
static String makeInflightIndexFileName(String instant) {
|
||||
return StringUtils.join(instant, HoodieTimeline.INFLIGHT_INDEX_COMMIT_EXTENSION);
|
||||
}
|
||||
|
||||
static String makeRequestedIndexFileName(String instant) {
|
||||
return StringUtils.join(instant, HoodieTimeline.REQUESTED_INDEX_COMMIT_EXTENSION);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -22,6 +22,8 @@ import org.apache.hudi.avro.HoodieAvroUtils;
|
||||
import org.apache.hudi.avro.model.HoodieCleanMetadata;
|
||||
import org.apache.hudi.avro.model.HoodieCleanerPlan;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionPlan;
|
||||
import org.apache.hudi.avro.model.HoodieIndexCommitMetadata;
|
||||
import org.apache.hudi.avro.model.HoodieIndexPlan;
|
||||
import org.apache.hudi.avro.model.HoodieInstantInfo;
|
||||
import org.apache.hudi.avro.model.HoodieReplaceCommitMetadata;
|
||||
import org.apache.hudi.avro.model.HoodieRequestedReplaceMetadata;
|
||||
@@ -137,6 +139,14 @@ public class TimelineMetadataUtils {
|
||||
return serializeAvroMetadata(clusteringPlan, HoodieRequestedReplaceMetadata.class);
|
||||
}
|
||||
|
||||
public static Option<byte[]> serializeIndexPlan(HoodieIndexPlan indexPlan) throws IOException {
|
||||
return serializeAvroMetadata(indexPlan, HoodieIndexPlan.class);
|
||||
}
|
||||
|
||||
public static Option<byte[]> serializeIndexCommitMetadata(HoodieIndexCommitMetadata indexCommitMetadata) throws IOException {
|
||||
return serializeAvroMetadata(indexCommitMetadata, HoodieIndexCommitMetadata.class);
|
||||
}
|
||||
|
||||
public static <T extends SpecificRecordBase> Option<byte[]> serializeAvroMetadata(T metadata, Class<T> clazz)
|
||||
throws IOException {
|
||||
DatumWriter<T> datumWriter = new SpecificDatumWriter<>(clazz);
|
||||
@@ -180,6 +190,14 @@ public class TimelineMetadataUtils {
|
||||
return deserializeAvroMetadata(bytes, HoodieReplaceCommitMetadata.class);
|
||||
}
|
||||
|
||||
public static HoodieIndexPlan deserializeIndexPlan(byte[] bytes) throws IOException {
|
||||
return deserializeAvroMetadata(bytes, HoodieIndexPlan.class);
|
||||
}
|
||||
|
||||
public static HoodieIndexCommitMetadata deserializeIndexCommitMetadata(byte[] bytes) throws IOException {
|
||||
return deserializeAvroMetadata(bytes, HoodieIndexCommitMetadata.class);
|
||||
}
|
||||
|
||||
public static <T extends SpecificRecordBase> T deserializeAvroMetadata(byte[] bytes, Class<T> clazz)
|
||||
throws IOException {
|
||||
DatumReader<T> reader = new SpecificDatumReader<>(clazz);
|
||||
|
||||
@@ -18,9 +18,9 @@
|
||||
|
||||
package org.apache.hudi.common.table.view;
|
||||
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hudi.common.model.BootstrapBaseFileMapping;
|
||||
import org.apache.hudi.common.model.CompactionOperation;
|
||||
import org.apache.hudi.common.model.FileSlice;
|
||||
import org.apache.hudi.common.model.HoodieFileGroup;
|
||||
import org.apache.hudi.common.model.HoodieFileGroupId;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
@@ -29,6 +29,8 @@ import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.ValidationUtils;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
@@ -358,6 +360,19 @@ public class HoodieTableFileSystemView extends IncrementalTimelineSyncFileSystem
|
||||
return Option.ofNullable(fgIdToReplaceInstants.get(fileGroupId));
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the latest file slices for a given partition including the inflight ones.
|
||||
*
|
||||
* @param partitionPath
|
||||
* @return Stream of latest {@link FileSlice} in the partition path.
|
||||
*/
|
||||
public Stream<FileSlice> fetchLatestFileSlicesIncludingInflight(String partitionPath) {
|
||||
return fetchAllStoredFileGroups(partitionPath)
|
||||
.map(HoodieFileGroup::getLatestFileSlicesIncludingInflight)
|
||||
.filter(Option::isPresent)
|
||||
.map(Option::get);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() {
|
||||
super.close();
|
||||
|
||||
@@ -20,12 +20,24 @@ package org.apache.hudi.common.util;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
import java.util.function.Function;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
/**
|
||||
* Simple utility for operations on strings.
|
||||
*/
|
||||
public class StringUtils {
|
||||
|
||||
public static final String EMPTY_STRING = "";
|
||||
private static final Function<String, Set<String>> STRING_TO_SET = (str) -> Stream.of(str.split(","))
|
||||
.map(String::trim).filter(s -> !s.isEmpty()).collect(Collectors.toSet());
|
||||
private static final Function<String, List<String>> STRING_TO_LIST = (str) -> Stream.of(str.split(","))
|
||||
.map(String::trim).filter(s -> !s.isEmpty()).collect(Collectors.toList());
|
||||
|
||||
/**
|
||||
* <p>
|
||||
@@ -46,7 +58,7 @@ public class StringUtils {
|
||||
* </pre>
|
||||
*/
|
||||
public static <T> String join(final String... elements) {
|
||||
return join(elements, "");
|
||||
return join(elements, EMPTY_STRING);
|
||||
}
|
||||
|
||||
public static <T> String joinUsingDelim(String delim, final String... elements) {
|
||||
@@ -100,4 +112,24 @@ public class StringUtils {
|
||||
private static boolean stringIsNullOrEmpty(@Nullable String string) {
|
||||
return string == null || string.isEmpty();
|
||||
}
|
||||
|
||||
/**
|
||||
* Converts the input string, delimited by comma, to a set of strings.
|
||||
*
|
||||
* @param input
|
||||
* @return
|
||||
*/
|
||||
public static Set<String> toSet(@Nullable String input) {
|
||||
return isNullOrEmpty(input) ? new HashSet<>() : STRING_TO_SET.apply(input);
|
||||
}
|
||||
|
||||
/**
|
||||
* Converts the input string, delimited by comma, to a list of strings.
|
||||
*
|
||||
* @param input
|
||||
* @return
|
||||
*/
|
||||
public static List<String> toList(@Nullable String input) {
|
||||
return isNullOrEmpty(input) ? new ArrayList<>() : STRING_TO_LIST.apply(input);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -284,7 +284,7 @@ public abstract class BaseTableMetadata implements HoodieTableMetadata {
|
||||
|
||||
List<String> partitions = Collections.emptyList();
|
||||
if (hoodieRecord.isPresent()) {
|
||||
mayBeHandleSpuriousDeletes(hoodieRecord, "\"all partitions\"");
|
||||
handleSpuriousDeletes(hoodieRecord, "\"all partitions\"");
|
||||
partitions = hoodieRecord.get().getData().getFilenames();
|
||||
// Partition-less tables have a single empty partition
|
||||
if (partitions.contains(NON_PARTITIONED_NAME)) {
|
||||
@@ -315,7 +315,7 @@ public abstract class BaseTableMetadata implements HoodieTableMetadata {
|
||||
|
||||
FileStatus[] statuses = {};
|
||||
if (hoodieRecord.isPresent()) {
|
||||
mayBeHandleSpuriousDeletes(hoodieRecord, partitionName);
|
||||
handleSpuriousDeletes(hoodieRecord, partitionName);
|
||||
statuses = hoodieRecord.get().getData().getFileStatuses(hadoopConf.get(), partitionPath);
|
||||
}
|
||||
|
||||
@@ -350,7 +350,7 @@ public abstract class BaseTableMetadata implements HoodieTableMetadata {
|
||||
|
||||
for (Pair<String, Option<HoodieRecord<HoodieMetadataPayload>>> entry: partitionsFileStatus) {
|
||||
if (entry.getValue().isPresent()) {
|
||||
mayBeHandleSpuriousDeletes(entry.getValue(), entry.getKey());
|
||||
handleSpuriousDeletes(entry.getValue(), entry.getKey());
|
||||
result.put(partitionInfo.get(entry.getKey()).toString(), entry.getValue().get().getData().getFileStatuses(hadoopConf.get(), partitionInfo.get(entry.getKey())));
|
||||
}
|
||||
}
|
||||
@@ -360,11 +360,11 @@ public abstract class BaseTableMetadata implements HoodieTableMetadata {
|
||||
}
|
||||
|
||||
/**
|
||||
* Maybe handle spurious deletes. Depending on config, throw an exception or log a warn msg.
|
||||
* Handle spurious deletes. Depending on config, throw an exception or log a warn msg.
|
||||
* @param hoodieRecord instance of {@link HoodieRecord} of interest.
|
||||
* @param partitionName partition name of interest.
|
||||
*/
|
||||
private void mayBeHandleSpuriousDeletes(Option<HoodieRecord<HoodieMetadataPayload>> hoodieRecord, String partitionName) {
|
||||
private void handleSpuriousDeletes(Option<HoodieRecord<HoodieMetadataPayload>> hoodieRecord, String partitionName) {
|
||||
if (!hoodieRecord.get().getData().getDeletions().isEmpty()) {
|
||||
if (metadataConfig.ignoreSpuriousDeletes()) {
|
||||
LOG.warn("Metadata record for " + partitionName + " encountered some files to be deleted which was not added before. "
|
||||
|
||||
@@ -74,6 +74,7 @@ import java.util.LinkedList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
import java.util.Set;
|
||||
import java.util.function.BiFunction;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
@@ -121,6 +122,39 @@ public class HoodieTableMetadataUtil {
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Deletes the metadata partition from the file system.
|
||||
*
|
||||
* @param basePath - base path of the dataset
|
||||
* @param context - instance of {@link HoodieEngineContext}
|
||||
* @param partitionType - {@link MetadataPartitionType} of the partition to delete
|
||||
*/
|
||||
public static void deleteMetadataPartition(String basePath, HoodieEngineContext context, MetadataPartitionType partitionType) {
|
||||
final String metadataTablePath = HoodieTableMetadata.getMetadataTableBasePath(basePath);
|
||||
FileSystem fs = FSUtils.getFs(metadataTablePath, context.getHadoopConf().get());
|
||||
try {
|
||||
fs.delete(new Path(metadataTablePath, partitionType.getPartitionPath()), true);
|
||||
} catch (Exception e) {
|
||||
throw new HoodieMetadataException(String.format("Failed to remove metadata partition %s from path %s", partitionType, metadataTablePath), e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Check if the given metadata partition exists.
|
||||
*
|
||||
* @param basePath base path of the dataset
|
||||
* @param context instance of {@link HoodieEngineContext}.
|
||||
*/
|
||||
public static boolean metadataPartitionExists(String basePath, HoodieEngineContext context, MetadataPartitionType partitionType) {
|
||||
final String metadataTablePath = HoodieTableMetadata.getMetadataTableBasePath(basePath);
|
||||
FileSystem fs = FSUtils.getFs(metadataTablePath, context.getHadoopConf().get());
|
||||
try {
|
||||
return fs.exists(new Path(metadataTablePath, partitionType.getPartitionPath()));
|
||||
} catch (Exception e) {
|
||||
throw new HoodieIOException(String.format("Failed to check metadata partition %s exists.", partitionType.getPartitionPath()));
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Convert commit action to metadata records for the enabled partition types.
|
||||
*
|
||||
@@ -885,6 +919,24 @@ public class HoodieTableMetadataUtil {
|
||||
return fileSliceStream.sorted(Comparator.comparing(FileSlice::getFileId)).collect(Collectors.toList());
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the latest file slices for a given partition including the inflight ones.
|
||||
*
|
||||
* @param metaClient - instance of {@link HoodieTableMetaClient}
|
||||
* @param fileSystemView - hoodie table file system view, which will be fetched from meta client if not already present
|
||||
* @param partition - name of the partition whose file groups are to be loaded
|
||||
* @return
|
||||
*/
|
||||
public static List<FileSlice> getPartitionLatestFileSlicesIncludingInflight(HoodieTableMetaClient metaClient,
|
||||
Option<HoodieTableFileSystemView> fileSystemView,
|
||||
String partition) {
|
||||
HoodieTableFileSystemView fsView = fileSystemView.orElse(getFileSystemView(metaClient));
|
||||
Stream<FileSlice> fileSliceStream = fsView.fetchLatestFileSlicesIncludingInflight(partition);
|
||||
return fileSliceStream
|
||||
.sorted(Comparator.comparing(FileSlice::getFileId))
|
||||
.collect(Collectors.toList());
|
||||
}
|
||||
|
||||
public static HoodieData<HoodieRecord> convertMetadataToColumnStatsRecords(HoodieCommitMetadata commitMetadata,
|
||||
HoodieEngineContext engineContext,
|
||||
MetadataRecordsGenerationParams recordsGenerationParams) {
|
||||
@@ -900,8 +952,8 @@ public class HoodieTableMetadataUtil {
|
||||
Option.ofNullable(commitMetadata.getMetadata(HoodieCommitMetadata.SCHEMA_KEY))
|
||||
.flatMap(writerSchemaStr ->
|
||||
isNullOrEmpty(writerSchemaStr)
|
||||
? Option.empty()
|
||||
: Option.of(new Schema.Parser().parse(writerSchemaStr)));
|
||||
? Option.empty()
|
||||
: Option.of(new Schema.Parser().parse(writerSchemaStr)));
|
||||
|
||||
HoodieTableMetaClient dataTableMetaClient = recordsGenerationParams.getDataMetaClient();
|
||||
HoodieTableConfig tableConfig = dataTableMetaClient.getTableConfig();
|
||||
@@ -1062,18 +1114,18 @@ public class HoodieTableMetadataUtil {
|
||||
* Aggregates column stats for each field.
|
||||
*
|
||||
* @param record - current record
|
||||
* @param schema - write schema
|
||||
* @param fields - fields for which stats will be aggregated
|
||||
* @param columnToStats - map of column to map of each stat and its value which gets updates in this method
|
||||
* @param consistentLogicalTimestampEnabled - flag to deal with logical timestamp type when getting column value
|
||||
*/
|
||||
public static void aggregateColumnStats(IndexedRecord record, Schema schema,
|
||||
public static void aggregateColumnStats(IndexedRecord record, List<Schema.Field> fields,
|
||||
Map<String, Map<String, Object>> columnToStats,
|
||||
boolean consistentLogicalTimestampEnabled) {
|
||||
if (!(record instanceof GenericRecord)) {
|
||||
throw new HoodieIOException("Record is not a generic type to get column range metadata!");
|
||||
}
|
||||
|
||||
schema.getFields().forEach(field -> {
|
||||
fields.forEach(field -> {
|
||||
Map<String, Object> columnStats = columnToStats.getOrDefault(field.name(), new HashMap<>());
|
||||
final String fieldVal = getNestedFieldValAsString((GenericRecord) record, field.name(), true, consistentLogicalTimestampEnabled);
|
||||
// update stats
|
||||
@@ -1114,4 +1166,18 @@ public class HoodieTableMetadataUtil {
|
||||
throw new HoodieException("Failed to get latest columns for " + dataTableMetaClient.getBasePath(), e);
|
||||
}
|
||||
}
|
||||
|
||||
public static Set<String> getInflightMetadataPartitions(HoodieTableConfig tableConfig) {
|
||||
return StringUtils.toSet(tableConfig.getMetadataPartitionsInflight());
|
||||
}
|
||||
|
||||
public static Set<String> getCompletedMetadataPartitions(HoodieTableConfig tableConfig) {
|
||||
return StringUtils.toSet(tableConfig.getMetadataPartitions());
|
||||
}
|
||||
|
||||
public static Set<String> getInflightAndCompletedMetadataPartitions(HoodieTableConfig tableConfig) {
|
||||
Set<String> inflightAndCompletedPartitions = getInflightMetadataPartitions(tableConfig);
|
||||
inflightAndCompletedPartitions.addAll(getCompletedMetadataPartitions(tableConfig));
|
||||
return inflightAndCompletedPartitions;
|
||||
}
|
||||
}
|
||||
|
||||
@@ -35,15 +35,19 @@ public class MetadataRecordsGenerationParams implements Serializable {
|
||||
private final int bloomIndexParallelism;
|
||||
private final boolean isAllColumnStatsIndexEnabled;
|
||||
private final int columnStatsIndexParallelism;
|
||||
private final List<String> columnsToIndex;
|
||||
private final List<String> bloomSecondaryKeys;
|
||||
|
||||
MetadataRecordsGenerationParams(HoodieTableMetaClient dataMetaClient, List<MetadataPartitionType> enabledPartitionTypes, String bloomFilterType, int bloomIndexParallelism,
|
||||
boolean isAllColumnStatsIndexEnabled, int columnStatsIndexParallelism) {
|
||||
boolean isAllColumnStatsIndexEnabled, int columnStatsIndexParallelism, List<String> columnsToIndex, List<String> bloomSecondaryKeys) {
|
||||
this.dataMetaClient = dataMetaClient;
|
||||
this.enabledPartitionTypes = enabledPartitionTypes;
|
||||
this.bloomFilterType = bloomFilterType;
|
||||
this.bloomIndexParallelism = bloomIndexParallelism;
|
||||
this.isAllColumnStatsIndexEnabled = isAllColumnStatsIndexEnabled;
|
||||
this.columnStatsIndexParallelism = columnStatsIndexParallelism;
|
||||
this.columnsToIndex = columnsToIndex;
|
||||
this.bloomSecondaryKeys = bloomSecondaryKeys;
|
||||
}
|
||||
|
||||
public HoodieTableMetaClient getDataMetaClient() {
|
||||
@@ -69,4 +73,12 @@ public class MetadataRecordsGenerationParams implements Serializable {
|
||||
public int getColumnStatsIndexParallelism() {
|
||||
return columnStatsIndexParallelism;
|
||||
}
|
||||
|
||||
public List<String> getColumnsToIndex() {
|
||||
return columnsToIndex;
|
||||
}
|
||||
|
||||
public List<String> getBloomSecondaryKeys() {
|
||||
return bloomSecondaryKeys;
|
||||
}
|
||||
}
|
||||
|
||||
@@ -199,6 +199,46 @@ public class TestHoodieActiveTimeline extends HoodieCommonTestHarness {
|
||||
assertTrue(activeCommitTimeline.isBeforeTimelineStarts("00"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetContiguousCompletedWriteTimeline() {
|
||||
// a mock timeline with holes
|
||||
timeline = new MockHoodieTimeline(Stream.of("01", "03", "05", "07", "13", "15", "17"),
|
||||
Stream.of("09", "11", "19"));
|
||||
assertTrue(timeline.getContiguousCompletedWriteTimeline().lastInstant().isPresent());
|
||||
assertEquals("07", timeline.getContiguousCompletedWriteTimeline().lastInstant().get().getTimestamp());
|
||||
|
||||
// add some instants where two are inflight and one of them (instant8 below) is not part of write timeline
|
||||
HoodieInstant instant1 = new HoodieInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "1");
|
||||
HoodieInstant instant2 = new HoodieInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "2");
|
||||
HoodieInstant instant3 = new HoodieInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "3");
|
||||
HoodieInstant instant4 = new HoodieInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "4");
|
||||
HoodieInstant instant5 = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, "5");
|
||||
HoodieInstant instant6 = new HoodieInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "6");
|
||||
HoodieInstant instant7 = new HoodieInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "7");
|
||||
HoodieInstant instant8 = new HoodieInstant(true, HoodieTimeline.RESTORE_ACTION, "8");
|
||||
|
||||
timeline = new HoodieActiveTimeline(metaClient);
|
||||
timeline.createNewInstant(instant1);
|
||||
timeline.createNewInstant(instant2);
|
||||
timeline.createNewInstant(instant3);
|
||||
timeline.createNewInstant(instant4);
|
||||
timeline.createNewInstant(instant5);
|
||||
timeline.createNewInstant(instant6);
|
||||
timeline.createNewInstant(instant7);
|
||||
timeline.createNewInstant(instant8);
|
||||
timeline.setInstants(Stream.of(instant1, instant2, instant3, instant4, instant5, instant6, instant7, instant8).collect(Collectors.toList()));
|
||||
|
||||
assertTrue(timeline.getContiguousCompletedWriteTimeline().lastInstant().isPresent());
|
||||
assertEquals(instant4.getTimestamp(), timeline.getContiguousCompletedWriteTimeline().lastInstant().get().getTimestamp());
|
||||
// transition both inflight instants to complete
|
||||
timeline.saveAsComplete(new HoodieInstant(true, instant5.getAction(), instant5.getTimestamp()), Option.empty());
|
||||
timeline.saveAsComplete(new HoodieInstant(true, instant8.getAction(), instant8.getTimestamp()), Option.empty());
|
||||
timeline = timeline.reload();
|
||||
// instant8 in not considered in write timeline, so last completed instant in timeline should be instant7
|
||||
assertTrue(timeline.getContiguousCompletedWriteTimeline().lastInstant().isPresent());
|
||||
assertEquals(instant7.getTimestamp(), timeline.getContiguousCompletedWriteTimeline().lastInstant().get().getTimestamp());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTimelineGetOperations() {
|
||||
List<HoodieInstant> allInstants = getAllInstants();
|
||||
@@ -218,20 +258,19 @@ public class TestHoodieActiveTimeline extends HoodieCommonTestHarness {
|
||||
|
||||
// Test that various types of getXXX operations from HoodieActiveTimeline
|
||||
// return the correct set of Instant
|
||||
checkTimeline.accept(timeline.getCommitsTimeline(),
|
||||
CollectionUtils.createSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION, HoodieTimeline.REPLACE_COMMIT_ACTION));
|
||||
checkTimeline.accept(timeline.getWriteTimeline(),
|
||||
CollectionUtils.createSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION, HoodieTimeline.COMPACTION_ACTION, HoodieTimeline.REPLACE_COMMIT_ACTION));
|
||||
checkTimeline.accept(timeline.getCommitsTimeline(), CollectionUtils.createSet(
|
||||
HoodieTimeline.COMMIT_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION, HoodieTimeline.REPLACE_COMMIT_ACTION));
|
||||
checkTimeline.accept(timeline.getWriteTimeline(), CollectionUtils.createSet(
|
||||
HoodieTimeline.COMMIT_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION, HoodieTimeline.COMPACTION_ACTION, HoodieTimeline.REPLACE_COMMIT_ACTION));
|
||||
checkTimeline.accept(timeline.getCommitTimeline(), CollectionUtils.createSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.REPLACE_COMMIT_ACTION));
|
||||
checkTimeline.accept(timeline.getDeltaCommitTimeline(), Collections.singleton(HoodieTimeline.DELTA_COMMIT_ACTION));
|
||||
checkTimeline.accept(timeline.getCleanerTimeline(), Collections.singleton(HoodieTimeline.CLEAN_ACTION));
|
||||
checkTimeline.accept(timeline.getRollbackTimeline(), Collections.singleton(HoodieTimeline.ROLLBACK_ACTION));
|
||||
checkTimeline.accept(timeline.getRestoreTimeline(), Collections.singleton(HoodieTimeline.RESTORE_ACTION));
|
||||
checkTimeline.accept(timeline.getSavePointTimeline(), Collections.singleton(HoodieTimeline.SAVEPOINT_ACTION));
|
||||
checkTimeline.accept(timeline.getAllCommitsTimeline(),
|
||||
CollectionUtils.createSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION,
|
||||
HoodieTimeline.CLEAN_ACTION, HoodieTimeline.COMPACTION_ACTION, HoodieTimeline.REPLACE_COMMIT_ACTION,
|
||||
HoodieTimeline.SAVEPOINT_ACTION, HoodieTimeline.ROLLBACK_ACTION));
|
||||
checkTimeline.accept(timeline.getAllCommitsTimeline(), CollectionUtils.createSet(
|
||||
HoodieTimeline.COMMIT_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION, HoodieTimeline.CLEAN_ACTION, HoodieTimeline.COMPACTION_ACTION,
|
||||
HoodieTimeline.REPLACE_COMMIT_ACTION, HoodieTimeline.SAVEPOINT_ACTION, HoodieTimeline.ROLLBACK_ACTION, HoodieTimeline.INDEXING_ACTION));
|
||||
|
||||
// Get some random Instants
|
||||
Random rand = new Random();
|
||||
|
||||
@@ -20,6 +20,12 @@ package org.apache.hudi.common.util;
|
||||
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertNotEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertNull;
|
||||
@@ -61,4 +67,20 @@ public class TestStringUtils {
|
||||
assertNotEquals(null, StringUtils.isNullOrEmpty("this is not empty"));
|
||||
assertTrue(StringUtils.isNullOrEmpty(""));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testStringToSet() {
|
||||
assertEquals(new HashSet<>(), StringUtils.toSet(null));
|
||||
assertEquals(new HashSet<>(), StringUtils.toSet(""));
|
||||
Set<String> expected = new HashSet<>(Arrays.asList("a", "b", "c"));
|
||||
assertEquals(expected, StringUtils.toSet("a,b, c"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testStringToList() {
|
||||
assertEquals(new ArrayList<>(), StringUtils.toList(null));
|
||||
assertEquals(new ArrayList<>(), StringUtils.toList(""));
|
||||
List<String> expected = Arrays.asList("a", "b", "c");
|
||||
assertEquals(expected, StringUtils.toList("a,b, c"));
|
||||
}
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user