1
0

[HUDI-1072] Introduce REPLACE top level action. Implement insert_overwrite operation on top of replace action (#2048)

This commit is contained in:
satishkotha
2020-09-29 17:04:25 -07:00
committed by GitHub
parent 32c9cad52c
commit a99e93bed5
60 changed files with 2129 additions and 380 deletions

View File

@@ -96,6 +96,14 @@
"name":"actionState",
"type":["null","string"],
"default": null
},
{
"name":"hoodieReplaceCommitMetadata",
"type":[
"null",
"HoodieReplaceCommitMetadata"
],
"default": null
}
]
}

View File

@@ -0,0 +1,32 @@
/*
* 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":"HoodieInstantInfo",
"fields":[
{
"name":"commitTime",
"type":"string"
},
{
"name":"action",
"type":"string"
}
]
}

View File

@@ -0,0 +1,64 @@
/*
* 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":"HoodieReplaceCommitMetadata",
"fields":[
{
"name":"partitionToWriteStats",
"type":["null", {
"type":"map",
"values":{
"type":"array",
"items":"HoodieWriteStat"
}
}],
"default": null
},
{
"name":"extraMetadata",
"type":["null", {
"type":"map",
"values":"string"
}],
"default": null
},
{
"name":"version",
"type":["int", "null"],
"default": 1
},
{
"name":"operationType",
"type": ["null","string"],
"default":null
},
{
"name":"partitionToReplaceFileIds",
"type":["null", {
"type":"map",
"values":{
"type":"array",
"items": "string"
}
}],
"default": null
}
]
}

View File

@@ -34,6 +34,16 @@
"name":"version",
"type":["int", "null"],
"default": 1
},
/* overlaps with 'instantsToRollback' field. Adding this to track action type for all the instants being rolled back. */
{
"name": "restoreInstantInfo",
"default": null,
"type": {
"type": "array",
"default": null,
"items": "HoodieInstantInfo"
}
}
]
}

View File

@@ -39,6 +39,16 @@
"name":"version",
"type":["int", "null"],
"default": 1
},
/* overlaps with 'commitsRollback' field. Adding this to track action type for all the instants being rolled back. */
{
"name": "instantsRollback",
"default": null,
"type": {
"type": "array",
"default": null,
"items": "HoodieInstantInfo"
}
}
]
}

View File

@@ -22,5 +22,6 @@ package org.apache.hudi.common.model;
* The supported action types.
*/
public enum ActionType {
commit, savepoint, compaction, clean, rollback
//TODO HUDI-1281 make deltacommit part of this
commit, savepoint, compaction, clean, rollback, replacecommit
}

View File

@@ -48,9 +48,9 @@ public class HoodieCommitMetadata implements Serializable {
protected Map<String, List<HoodieWriteStat>> partitionToWriteStats;
protected Boolean compacted;
private Map<String, String> extraMetadata;
protected Map<String, String> extraMetadata;
private WriteOperationType operationType = WriteOperationType.UNKNOWN;
protected WriteOperationType operationType = WriteOperationType.UNKNOWN;
// for ser/deser
public HoodieCommitMetadata() {
@@ -365,7 +365,9 @@ public class HoodieCommitMetadata implements Serializable {
@Override
public String toString() {
return "HoodieCommitMetadata{partitionToWriteStats=" + partitionToWriteStats + ", compacted=" + compacted
+ ", extraMetadata=" + extraMetadata + '}';
return "HoodieCommitMetadata{" + "partitionToWriteStats=" + partitionToWriteStats
+ ", compacted=" + compacted
+ ", extraMetadata=" + extraMetadata
+ ", operationType=" + operationType + '}';
}
}

View File

@@ -0,0 +1,142 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common.model;
import com.fasterxml.jackson.annotation.JsonAutoDetect;
import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
import com.fasterxml.jackson.annotation.PropertyAccessor;
import com.fasterxml.jackson.databind.DeserializationFeature;
import com.fasterxml.jackson.databind.ObjectMapper;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import java.io.IOException;
import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
/**
* All the metadata that gets stored along with a commit.
*/
@JsonIgnoreProperties(ignoreUnknown = true)
public class HoodieReplaceCommitMetadata extends HoodieCommitMetadata {
private static final Logger LOG = LogManager.getLogger(HoodieReplaceCommitMetadata.class);
protected Map<String, List<String>> partitionToReplaceFileIds;
// for ser/deser
public HoodieReplaceCommitMetadata() {
this(false);
}
public HoodieReplaceCommitMetadata(boolean compacted) {
super(compacted);
partitionToReplaceFileIds = new HashMap<>();
}
public void setPartitionToReplaceFileIds(Map<String, List<String>> partitionToReplaceFileIds) {
this.partitionToReplaceFileIds = partitionToReplaceFileIds;
}
public void addReplaceFileId(String partitionPath, String fileId) {
if (!partitionToReplaceFileIds.containsKey(partitionPath)) {
partitionToReplaceFileIds.put(partitionPath, new ArrayList<>());
}
partitionToReplaceFileIds.get(partitionPath).add(fileId);
}
public List<String> getReplaceFileIds(String partitionPath) {
return partitionToReplaceFileIds.get(partitionPath);
}
public Map<String, List<String>> getPartitionToReplaceFileIds() {
return partitionToReplaceFileIds;
}
@Override
public String toJsonString() throws IOException {
if (partitionToWriteStats.containsKey(null)) {
LOG.info("partition path is null for " + partitionToWriteStats.get(null));
partitionToWriteStats.remove(null);
}
if (partitionToReplaceFileIds.containsKey(null)) {
LOG.info("partition path is null for " + partitionToReplaceFileIds.get(null));
partitionToReplaceFileIds.remove(null);
}
return getObjectMapper().writerWithDefaultPrettyPrinter().writeValueAsString(this);
}
public static <T> T fromJsonString(String jsonStr, Class<T> clazz) throws Exception {
if (jsonStr == null || jsonStr.isEmpty()) {
// For empty commit file (no data or somethings bad happen).
return clazz.newInstance();
}
return getObjectMapper().readValue(jsonStr, clazz);
}
@Override
public boolean equals(Object o) {
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
HoodieReplaceCommitMetadata that = (HoodieReplaceCommitMetadata) o;
if (!partitionToWriteStats.equals(that.partitionToWriteStats)) {
return false;
}
return compacted.equals(that.compacted);
}
@Override
public int hashCode() {
int result = partitionToWriteStats.hashCode();
result = 31 * result + compacted.hashCode();
return result;
}
public static <T> T fromBytes(byte[] bytes, Class<T> clazz) throws IOException {
try {
return fromJsonString(new String(bytes, StandardCharsets.UTF_8), clazz);
} catch (Exception e) {
throw new IOException("unable to read commit metadata", e);
}
}
protected static ObjectMapper getObjectMapper() {
ObjectMapper mapper = new ObjectMapper();
mapper.disable(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES);
mapper.setVisibility(PropertyAccessor.FIELD, JsonAutoDetect.Visibility.ANY);
return mapper;
}
@Override
public String toString() {
return "HoodieReplaceMetadata{" + "partitionToWriteStats=" + partitionToWriteStats
+ ", partitionToReplaceFileIds=" + partitionToReplaceFileIds
+ ", compacted=" + compacted
+ ", extraMetadata=" + extraMetadata
+ ", operationType=" + operationType + '}';
}
}

View File

@@ -38,6 +38,8 @@ public enum WriteOperationType {
// delete
DELETE("delete"),
BOOTSTRAP("bootstrap"),
// insert overwrite
INSERT_OVERWRITE("insert_overwrite"),
// used for old version
UNKNOWN("unknown");
@@ -66,6 +68,8 @@ public enum WriteOperationType {
return BULK_INSERT_PREPPED;
case "delete":
return DELETE;
case "insert_overwrite":
return INSERT_OVERWRITE;
default:
throw new HoodieException("Invalid value of Type.");
}

View File

@@ -31,6 +31,7 @@ import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.common.table.timeline.TimelineLayout;
import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion;
import org.apache.hudi.common.util.CommitUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.ValidationUtils;
import org.apache.hudi.exception.HoodieException;
@@ -524,14 +525,7 @@ public class HoodieTableMetaClient implements Serializable {
* Gets the commit action type.
*/
public String getCommitActionType() {
switch (this.getTableType()) {
case COPY_ON_WRITE:
return HoodieActiveTimeline.COMMIT_ACTION;
case MERGE_ON_READ:
return HoodieActiveTimeline.DELTA_COMMIT_ACTION;
default:
throw new HoodieException("Could not commit on unknown table type " + this.getTableType());
}
return CommitUtils.getCommitActionType(this.getTableType());
}
/**

View File

@@ -65,7 +65,8 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
COMMIT_EXTENSION, INFLIGHT_COMMIT_EXTENSION, REQUESTED_COMMIT_EXTENSION, DELTA_COMMIT_EXTENSION,
INFLIGHT_DELTA_COMMIT_EXTENSION, REQUESTED_DELTA_COMMIT_EXTENSION, SAVEPOINT_EXTENSION,
INFLIGHT_SAVEPOINT_EXTENSION, CLEAN_EXTENSION, REQUESTED_CLEAN_EXTENSION, INFLIGHT_CLEAN_EXTENSION,
INFLIGHT_COMPACTION_EXTENSION, REQUESTED_COMPACTION_EXTENSION, INFLIGHT_RESTORE_EXTENSION, RESTORE_EXTENSION));
INFLIGHT_COMPACTION_EXTENSION, REQUESTED_COMPACTION_EXTENSION, INFLIGHT_RESTORE_EXTENSION, RESTORE_EXTENSION,
REQUESTED_REPLACE_COMMIT_EXTENSION, INFLIGHT_REPLACE_COMMIT_EXTENSION, REPLACE_COMMIT_EXTENSION));
private static final Logger LOG = LogManager.getLogger(HoodieActiveTimeline.class);
protected HoodieTableMetaClient metaClient;
@@ -304,6 +305,22 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
return inflight;
}
/**
* Transition Clean State from inflight to Committed.
*
* @param inflightInstant Inflight instant
* @param data Extra Metadata
* @return commit instant
*/
public HoodieInstant transitionReplaceInflightToComplete(HoodieInstant inflightInstant, Option<byte[]> data) {
ValidationUtils.checkArgument(inflightInstant.getAction().equals(HoodieTimeline.REPLACE_COMMIT_ACTION));
ValidationUtils.checkArgument(inflightInstant.isInflight());
HoodieInstant commitInstant = new HoodieInstant(State.COMPLETED, REPLACE_COMMIT_ACTION, inflightInstant.getTimestamp());
// Then write to timeline
transitionState(inflightInstant, commitInstant, data);
return commitInstant;
}
private void transitionState(HoodieInstant fromInstant, HoodieInstant toInstant, Option<byte[]> data) {
transitionState(fromInstant, toInstant, data, false);
}

View File

@@ -27,7 +27,6 @@ import org.apache.hudi.exception.HoodieException;
import java.io.Serializable;
import java.security.MessageDigest;
import java.security.NoSuchAlgorithmException;
import java.util.Collections;
import java.util.List;
import java.util.Set;
import java.util.function.Function;
@@ -109,10 +108,16 @@ public class HoodieDefaultTimeline implements HoodieTimeline {
@Override
public HoodieDefaultTimeline getCommitsAndCompactionTimeline() {
Set<String> validActions = CollectionUtils.createSet(COMMIT_ACTION, DELTA_COMMIT_ACTION, COMPACTION_ACTION);
Set<String> validActions = CollectionUtils.createSet(COMMIT_ACTION, DELTA_COMMIT_ACTION, COMPACTION_ACTION, REPLACE_COMMIT_ACTION);
return new HoodieDefaultTimeline(instants.stream().filter(s -> validActions.contains(s.getAction())), details);
}
@Override
public HoodieTimeline getCompletedReplaceTimeline() {
return new HoodieDefaultTimeline(
instants.stream().filter(s -> s.getAction().equals(REPLACE_COMMIT_ACTION)).filter(s -> s.isCompleted()), details);
}
@Override
public HoodieTimeline filterPendingCompactionTimeline() {
return new HoodieDefaultTimeline(
@@ -155,7 +160,7 @@ public class HoodieDefaultTimeline implements HoodieTimeline {
* Get all instants (commits, delta commits) that produce new data, in the active timeline.
*/
public HoodieTimeline getCommitsTimeline() {
return getTimelineOfActions(CollectionUtils.createSet(COMMIT_ACTION, DELTA_COMMIT_ACTION));
return getTimelineOfActions(CollectionUtils.createSet(COMMIT_ACTION, DELTA_COMMIT_ACTION, REPLACE_COMMIT_ACTION));
}
/**
@@ -164,14 +169,14 @@ public class HoodieDefaultTimeline implements HoodieTimeline {
*/
public HoodieTimeline getAllCommitsTimeline() {
return getTimelineOfActions(CollectionUtils.createSet(COMMIT_ACTION, DELTA_COMMIT_ACTION,
CLEAN_ACTION, COMPACTION_ACTION, SAVEPOINT_ACTION, ROLLBACK_ACTION));
CLEAN_ACTION, COMPACTION_ACTION, SAVEPOINT_ACTION, ROLLBACK_ACTION, REPLACE_COMMIT_ACTION));
}
/**
* Get only pure commits (inflight and completed) in the active timeline.
*/
public HoodieTimeline getCommitTimeline() {
return getTimelineOfActions(Collections.singleton(COMMIT_ACTION));
return getTimelineOfActions(CollectionUtils.createSet(COMMIT_ACTION, REPLACE_COMMIT_ACTION));
}
/**

View File

@@ -166,6 +166,10 @@ public class HoodieInstant implements Serializable, Comparable<HoodieInstant> {
} else if (HoodieTimeline.RESTORE_ACTION.equals(action)) {
return isInflight() ? HoodieTimeline.makeInflightRestoreFileName(timestamp)
: HoodieTimeline.makeRestoreFileName(timestamp);
} else if (HoodieTimeline.REPLACE_COMMIT_ACTION.equals(action)) {
return isInflight() ? HoodieTimeline.makeInflightReplaceFileName(timestamp)
: isRequested() ? HoodieTimeline.makeRequestedReplaceFileName(timestamp)
: HoodieTimeline.makeReplaceFileName(timestamp);
}
throw new IllegalArgumentException("Cannot get file name for unknown action " + action);
}

View File

@@ -48,6 +48,7 @@ public interface HoodieTimeline extends Serializable {
String CLEAN_ACTION = "clean";
String ROLLBACK_ACTION = "rollback";
String SAVEPOINT_ACTION = "savepoint";
String REPLACE_COMMIT_ACTION = "replacecommit";
String INFLIGHT_EXTENSION = ".inflight";
// With Async Compaction, compaction instant can be in 3 states :
// (compaction-requested), (compaction-inflight), (completed)
@@ -57,7 +58,7 @@ public interface HoodieTimeline extends Serializable {
String[] VALID_ACTIONS_IN_TIMELINE = {COMMIT_ACTION, DELTA_COMMIT_ACTION,
CLEAN_ACTION, SAVEPOINT_ACTION, RESTORE_ACTION, ROLLBACK_ACTION,
COMPACTION_ACTION};
COMPACTION_ACTION, REPLACE_COMMIT_ACTION};
String COMMIT_EXTENSION = "." + COMMIT_ACTION;
String DELTA_COMMIT_EXTENSION = "." + DELTA_COMMIT_ACTION;
@@ -78,6 +79,9 @@ public interface HoodieTimeline extends Serializable {
String INFLIGHT_COMPACTION_EXTENSION = StringUtils.join(".", COMPACTION_ACTION, INFLIGHT_EXTENSION);
String INFLIGHT_RESTORE_EXTENSION = "." + RESTORE_ACTION + INFLIGHT_EXTENSION;
String RESTORE_EXTENSION = "." + RESTORE_ACTION;
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 INVALID_INSTANT_TS = "0";
@@ -133,6 +137,13 @@ public interface HoodieTimeline extends Serializable {
*/
HoodieTimeline getCommitsAndCompactionTimeline();
/**
* Timeline to just include replace instants that have valid (commit/deltacommit) actions.
*
* @return
*/
HoodieTimeline getCompletedReplaceTimeline();
/**
* Filter this timeline to just include requested and inflight compaction instants.
*
@@ -360,6 +371,18 @@ public interface HoodieTimeline extends Serializable {
return StringUtils.join(instant, HoodieTimeline.INFLIGHT_RESTORE_EXTENSION);
}
static String makeReplaceFileName(String instant) {
return StringUtils.join(instant, HoodieTimeline.REPLACE_COMMIT_EXTENSION);
}
static String makeInflightReplaceFileName(String instant) {
return StringUtils.join(instant, HoodieTimeline.INFLIGHT_REPLACE_COMMIT_EXTENSION);
}
static String makeRequestedReplaceFileName(String instant) {
return StringUtils.join(instant, HoodieTimeline.REQUESTED_REPLACE_COMMIT_EXTENSION);
}
static String makeDeltaFileName(String instantTime) {
return instantTime + HoodieTimeline.DELTA_COMMIT_EXTENSION;
}

View File

@@ -21,6 +21,7 @@ package org.apache.hudi.common.table.timeline;
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.HoodieInstantInfo;
import org.apache.hudi.avro.model.HoodieRestoreMetadata;
import org.apache.hudi.avro.model.HoodieRollbackMetadata;
import org.apache.hudi.avro.model.HoodieRollbackPartitionMetadata;
@@ -46,6 +47,7 @@ import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
public class TimelineMetadataUtils {
@@ -53,14 +55,16 @@ public class TimelineMetadataUtils {
public static HoodieRestoreMetadata convertRestoreMetadata(String startRestoreTime,
long durationInMs,
List<String> commits,
List<HoodieInstant> instants,
Map<String, List<HoodieRollbackMetadata>> instantToRollbackMetadata) {
return new HoodieRestoreMetadata(startRestoreTime, durationInMs, commits,
Collections.unmodifiableMap(instantToRollbackMetadata), DEFAULT_VERSION);
return new HoodieRestoreMetadata(startRestoreTime, durationInMs,
instants.stream().map(HoodieInstant::getTimestamp).collect(Collectors.toList()),
Collections.unmodifiableMap(instantToRollbackMetadata), DEFAULT_VERSION,
instants.stream().map(instant -> new HoodieInstantInfo(instant.getTimestamp(), instant.getAction())).collect(Collectors.toList()));
}
public static HoodieRollbackMetadata convertRollbackMetadata(String startRollbackTime, Option<Long> durationInMs,
List<String> commits, List<HoodieRollbackStat> rollbackStats) {
List<HoodieInstant> instants, List<HoodieRollbackStat> rollbackStats) {
Map<String, HoodieRollbackPartitionMetadata> partitionMetadataBuilder = new HashMap<>();
int totalDeleted = 0;
for (HoodieRollbackStat stat : rollbackStats) {
@@ -70,8 +74,10 @@ public class TimelineMetadataUtils {
totalDeleted += stat.getSuccessDeleteFiles().size();
}
return new HoodieRollbackMetadata(startRollbackTime, durationInMs.orElseGet(() -> -1L), totalDeleted, commits,
Collections.unmodifiableMap(partitionMetadataBuilder), DEFAULT_VERSION);
return new HoodieRollbackMetadata(startRollbackTime, durationInMs.orElseGet(() -> -1L), totalDeleted,
instants.stream().map(HoodieInstant::getTimestamp).collect(Collectors.toList()),
Collections.unmodifiableMap(partitionMetadataBuilder), DEFAULT_VERSION,
instants.stream().map(instant -> new HoodieInstantInfo(instant.getTimestamp(), instant.getAction())).collect(Collectors.toList()));
}
public static HoodieSavepointMetadata convertSavepointMetadata(String user, String comment,

View File

@@ -18,16 +18,19 @@
package org.apache.hudi.common.table.view;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.common.bootstrap.index.BootstrapIndex;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.BootstrapBaseFileMapping;
import org.apache.hudi.common.model.BootstrapFileMapping;
import org.apache.hudi.common.model.CompactionOperation;
import org.apache.hudi.common.model.BootstrapBaseFileMapping;
import org.apache.hudi.common.model.FileSlice;
import org.apache.hudi.common.model.HoodieBaseFile;
import org.apache.hudi.common.model.HoodieFileGroup;
import org.apache.hudi.common.model.HoodieFileGroupId;
import org.apache.hudi.common.model.HoodieLogFile;
import org.apache.hudi.common.model.HoodieReplaceCommitMetadata;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
@@ -37,15 +40,15 @@ 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.hudi.exception.HoodieIOException;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import java.io.IOException;
import java.io.Serializable;
import java.util.AbstractMap;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Comparator;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
@@ -58,6 +61,7 @@ import java.util.function.Predicate;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import static org.apache.hudi.common.table.timeline.HoodieTimeline.GREATER_THAN_OR_EQUALS;
import static org.apache.hudi.common.table.timeline.HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS;
/**
@@ -97,6 +101,7 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV
protected void init(HoodieTableMetaClient metaClient, HoodieTimeline visibleActiveTimeline) {
this.metaClient = metaClient;
refreshTimeline(visibleActiveTimeline);
resetFileGroupsReplaced(visibleCommitsAndCompactionTimeline);
this.bootstrapIndex = BootstrapIndex.getBootstrapIndex(metaClient);
// Load Pending Compaction Operations
resetPendingCompactionOperations(CompactionUtils.getAllPendingCompactionOperations(metaClient).values().stream()
@@ -196,6 +201,33 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV
return fileGroups;
}
/**
* Get replaced instant for each file group by looking at all commit instants.
*/
private void resetFileGroupsReplaced(HoodieTimeline timeline) {
HoodieTimer hoodieTimer = new HoodieTimer();
hoodieTimer.startTimer();
// for each REPLACE instant, get map of (partitionPath -> deleteFileGroup)
HoodieTimeline replacedTimeline = timeline.getCompletedReplaceTimeline();
Stream<Map.Entry<HoodieFileGroupId, HoodieInstant>> resultStream = replacedTimeline.getInstants().flatMap(instant -> {
try {
HoodieReplaceCommitMetadata replaceMetadata = HoodieReplaceCommitMetadata.fromBytes(metaClient.getActiveTimeline().getInstantDetails(instant).get(),
HoodieReplaceCommitMetadata.class);
// get replace instant mapping for each partition, fileId
return replaceMetadata.getPartitionToReplaceFileIds().entrySet().stream().flatMap(entry -> entry.getValue().stream().map(e ->
new AbstractMap.SimpleEntry<>(new HoodieFileGroupId(entry.getKey(), e), instant)));
} catch (IOException e) {
throw new HoodieIOException("error reading commit metadata for " + instant);
}
});
Map<HoodieFileGroupId, HoodieInstant> replacedFileGroups = resultStream.collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
resetReplacedFileGroups(replacedFileGroups);
LOG.info("Took " + hoodieTimer.endTimer() + " ms to read " + replacedTimeline.countInstants() + " instants, "
+ replacedFileGroups.size() + " replaced file groups");
}
/**
* Clears the partition Map and reset view states.
*/
@@ -385,6 +417,7 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV
String partitionPath = formatPartitionKey(partitionStr);
ensurePartitionLoadedCorrectly(partitionPath);
return fetchLatestBaseFiles(partitionPath)
.filter(df -> !isFileGroupReplaced(partitionPath, df.getFileId()))
.map(df -> addBootstrapBaseFileIfPresent(new HoodieFileGroupId(partitionPath, df.getFileId()), df));
} finally {
readLock.unlock();
@@ -408,6 +441,7 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV
String partitionPath = formatPartitionKey(partitionStr);
ensurePartitionLoadedCorrectly(partitionPath);
return fetchAllStoredFileGroups(partitionPath)
.filter(fileGroup -> !isFileGroupReplacedBeforeOrOn(fileGroup.getFileGroupId(), maxCommitTime))
.map(fileGroup -> Option.fromJavaOptional(fileGroup.getAllBaseFiles()
.filter(baseFile -> HoodieTimeline.compareTimestamps(baseFile.getCommitTime(), HoodieTimeline.LESSER_THAN_OR_EQUALS, maxCommitTime
))
@@ -425,10 +459,14 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV
readLock.lock();
String partitionPath = formatPartitionKey(partitionStr);
ensurePartitionLoadedCorrectly(partitionPath);
return fetchHoodieFileGroup(partitionPath, fileId).map(fileGroup -> fileGroup.getAllBaseFiles()
.filter(baseFile -> HoodieTimeline.compareTimestamps(baseFile.getCommitTime(), HoodieTimeline.EQUALS,
instantTime)).filter(df -> !isBaseFileDueToPendingCompaction(df)).findFirst().orElse(null))
.map(df -> addBootstrapBaseFileIfPresent(new HoodieFileGroupId(partitionPath, fileId), df));
if (isFileGroupReplacedBeforeOrOn(new HoodieFileGroupId(partitionPath, fileId), instantTime)) {
return Option.empty();
} else {
return fetchHoodieFileGroup(partitionPath, fileId).map(fileGroup -> fileGroup.getAllBaseFiles()
.filter(baseFile -> HoodieTimeline.compareTimestamps(baseFile.getCommitTime(), HoodieTimeline.EQUALS,
instantTime)).filter(df -> !isBaseFileDueToPendingCompaction(df)).findFirst().orElse(null))
.map(df -> addBootstrapBaseFileIfPresent(new HoodieFileGroupId(partitionPath, fileId), df));
}
} finally {
readLock.unlock();
}
@@ -443,8 +481,12 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV
readLock.lock();
String partitionPath = formatPartitionKey(partitionStr);
ensurePartitionLoadedCorrectly(partitionPath);
return fetchLatestBaseFile(partitionPath, fileId)
.map(df -> addBootstrapBaseFileIfPresent(new HoodieFileGroupId(partitionPath, fileId), df));
if (isFileGroupReplaced(partitionPath, fileId)) {
return Option.empty();
} else {
return fetchLatestBaseFile(partitionPath, fileId)
.map(df -> addBootstrapBaseFileIfPresent(new HoodieFileGroupId(partitionPath, fileId), df));
}
} finally {
readLock.unlock();
}
@@ -454,7 +496,9 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV
public final Stream<HoodieBaseFile> getLatestBaseFilesInRange(List<String> commitsToReturn) {
try {
readLock.lock();
return fetchAllStoredFileGroups().map(fileGroup -> Pair.of(fileGroup.getFileGroupId(), Option.fromJavaOptional(
return fetchAllStoredFileGroups()
.filter(fileGroup -> !isFileGroupReplacedBeforeAny(fileGroup.getFileGroupId(), commitsToReturn))
.map(fileGroup -> Pair.of(fileGroup.getFileGroupId(), Option.fromJavaOptional(
fileGroup.getAllBaseFiles().filter(baseFile -> commitsToReturn.contains(baseFile.getCommitTime())
&& !isBaseFileDueToPendingCompaction(baseFile)).findFirst()))).filter(p -> p.getValue().isPresent())
.map(p -> addBootstrapBaseFileIfPresent(p.getKey(), p.getValue().get()));
@@ -470,6 +514,7 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV
String partitionPath = formatPartitionKey(partitionStr);
ensurePartitionLoadedCorrectly(partitionPath);
return fetchAllBaseFiles(partitionPath)
.filter(df -> !isFileGroupReplaced(partitionPath, df.getFileId()))
.filter(df -> visibleCommitsAndCompactionTimeline.containsOrBeforeTimelineStarts(df.getCommitTime()))
.filter(df -> !isBaseFileDueToPendingCompaction(df))
.map(df -> addBootstrapBaseFileIfPresent(new HoodieFileGroupId(partitionPath, df.getFileId()), df));
@@ -484,7 +529,9 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV
readLock.lock();
String partitionPath = formatPartitionKey(partitionStr);
ensurePartitionLoadedCorrectly(partitionPath);
return fetchLatestFileSlices(partitionPath).map(this::filterBaseFileAfterPendingCompaction)
return fetchLatestFileSlices(partitionPath)
.filter(slice -> !isFileGroupReplaced(slice.getFileGroupId()))
.map(this::filterBaseFileAfterPendingCompaction)
.map(this::addBootstrapBaseFileIfPresent);
} finally {
readLock.unlock();
@@ -500,8 +547,12 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV
readLock.lock();
String partitionPath = formatPartitionKey(partitionStr);
ensurePartitionLoadedCorrectly(partitionPath);
Option<FileSlice> fs = fetchLatestFileSlice(partitionPath, fileId);
return fs.map(this::filterBaseFileAfterPendingCompaction).map(this::addBootstrapBaseFileIfPresent);
if (isFileGroupReplaced(partitionPath, fileId)) {
return Option.empty();
} else {
Option<FileSlice> fs = fetchLatestFileSlice(partitionPath, fileId);
return fs.map(this::filterBaseFileAfterPendingCompaction).map(this::addBootstrapBaseFileIfPresent);
}
} finally {
readLock.unlock();
}
@@ -513,17 +564,19 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV
readLock.lock();
String partitionPath = formatPartitionKey(partitionStr);
ensurePartitionLoadedCorrectly(partitionPath);
return fetchAllStoredFileGroups(partitionPath).map(fileGroup -> {
FileSlice fileSlice = fileGroup.getLatestFileSlice().get();
// if the file-group is under compaction, pick the latest before compaction instant time.
Option<Pair<String, CompactionOperation>> compactionWithInstantPair =
getPendingCompactionOperationWithInstant(fileSlice.getFileGroupId());
if (compactionWithInstantPair.isPresent()) {
String compactionInstantTime = compactionWithInstantPair.get().getLeft();
return fileGroup.getLatestFileSliceBefore(compactionInstantTime);
}
return Option.of(fileSlice);
}).map(Option::get).map(this::addBootstrapBaseFileIfPresent);
return fetchAllStoredFileGroups(partitionPath)
.filter(fg -> !isFileGroupReplaced(fg.getFileGroupId()))
.map(fileGroup -> {
FileSlice fileSlice = fileGroup.getLatestFileSlice().get();
// if the file-group is under compaction, pick the latest before compaction instant time.
Option<Pair<String, CompactionOperation>> compactionWithInstantPair =
getPendingCompactionOperationWithInstant(fileSlice.getFileGroupId());
if (compactionWithInstantPair.isPresent()) {
String compactionInstantTime = compactionWithInstantPair.get().getLeft();
return fileGroup.getLatestFileSliceBefore(compactionInstantTime);
}
return Option.of(fileSlice);
}).map(Option::get).map(this::addBootstrapBaseFileIfPresent);
} finally {
readLock.unlock();
}
@@ -536,7 +589,8 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV
readLock.lock();
String partitionPath = formatPartitionKey(partitionStr);
ensurePartitionLoadedCorrectly(partitionPath);
Stream<FileSlice> fileSliceStream = fetchLatestFileSlicesBeforeOrOn(partitionPath, maxCommitTime);
Stream<FileSlice> fileSliceStream = fetchLatestFileSlicesBeforeOrOn(partitionPath, maxCommitTime)
.filter(slice -> !isFileGroupReplacedBeforeOrOn(slice.getFileGroupId(), maxCommitTime));
if (includeFileSlicesInPendingCompaction) {
return fileSliceStream.map(this::filterBaseFileAfterPendingCompaction).map(this::addBootstrapBaseFileIfPresent);
} else {
@@ -554,14 +608,16 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV
readLock.lock();
String partition = formatPartitionKey(partitionStr);
ensurePartitionLoadedCorrectly(partition);
return fetchAllStoredFileGroups(partition).map(fileGroup -> {
Option<FileSlice> fileSlice = fileGroup.getLatestFileSliceBeforeOrOn(maxInstantTime);
// if the file-group is under construction, pick the latest before compaction instant time.
if (fileSlice.isPresent()) {
fileSlice = Option.of(fetchMergedFileSlice(fileGroup, fileSlice.get()));
}
return fileSlice;
}).filter(Option::isPresent).map(Option::get).map(this::addBootstrapBaseFileIfPresent);
return fetchAllStoredFileGroups(partition)
.filter(fg -> !isFileGroupReplacedBeforeOrOn(fg.getFileGroupId(), maxInstantTime))
.map(fileGroup -> {
Option<FileSlice> fileSlice = fileGroup.getLatestFileSliceBeforeOrOn(maxInstantTime);
// if the file-group is under construction, pick the latest before compaction instant time.
if (fileSlice.isPresent()) {
fileSlice = Option.of(fetchMergedFileSlice(fileGroup, fileSlice.get()));
}
return fileSlice;
}).filter(Option::isPresent).map(Option::get).map(this::addBootstrapBaseFileIfPresent);
} finally {
readLock.unlock();
}
@@ -571,7 +627,9 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV
public final Stream<FileSlice> getLatestFileSliceInRange(List<String> commitsToReturn) {
try {
readLock.lock();
return fetchLatestFileSliceInRange(commitsToReturn).map(this::addBootstrapBaseFileIfPresent);
return fetchLatestFileSliceInRange(commitsToReturn)
.filter(slice -> !isFileGroupReplacedBeforeAny(slice.getFileGroupId(), commitsToReturn))
.map(this::addBootstrapBaseFileIfPresent);
} finally {
readLock.unlock();
}
@@ -583,7 +641,7 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV
readLock.lock();
String partition = formatPartitionKey(partitionStr);
ensurePartitionLoadedCorrectly(partition);
return fetchAllFileSlices(partition).map(this::addBootstrapBaseFileIfPresent);
return fetchAllFileSlices(partition).filter(slice -> !isFileGroupReplaced(slice.getFileGroupId())).map(this::addBootstrapBaseFileIfPresent);
} finally {
readLock.unlock();
}
@@ -599,6 +657,10 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV
@Override
public final Stream<HoodieFileGroup> getAllFileGroups(String partitionStr) {
return getAllFileGroupsIncludingReplaced(partitionStr).filter(fg -> !isFileGroupReplaced(fg));
}
private Stream<HoodieFileGroup> getAllFileGroupsIncludingReplaced(final String partitionStr) {
try {
readLock.lock();
// Ensure there is consistency in handling trailing slash in partition-path. Always trim it which is what is done
@@ -611,6 +673,12 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV
}
}
@Override
public Stream<HoodieFileGroup> getReplacedFileGroupsBeforeOrOn(String maxCommitTime, String partitionPath) {
return getAllFileGroupsIncludingReplaced(partitionPath).filter(fg -> isFileGroupReplacedBeforeOrOn(fg.getFileGroupId(), maxCommitTime));
}
// Fetch APIs to be implemented by concrete sub-classes
/**
@@ -727,6 +795,26 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV
*/
abstract Stream<HoodieFileGroup> fetchAllStoredFileGroups();
/**
* Track instant time for file groups replaced.
*/
protected abstract void resetReplacedFileGroups(final Map<HoodieFileGroupId, HoodieInstant> replacedFileGroups);
/**
* Track instant time for new file groups replaced.
*/
protected abstract void addReplacedFileGroups(final Map<HoodieFileGroupId, HoodieInstant> replacedFileGroups);
/**
* Remove file groups that are replaced in any of the specified instants.
*/
protected abstract void removeReplacedFileIdsAtInstants(Set<String> instants);
/**
* Track instant time for file groups replaced.
*/
protected abstract Option<HoodieInstant> getReplaceInstant(final HoodieFileGroupId fileGroupId);
/**
* Check if the view is already closed.
*/
@@ -772,7 +860,9 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV
* Fetch latest base-files across all partitions.
*/
private Stream<HoodieBaseFile> fetchLatestBaseFiles() {
return fetchAllStoredFileGroups().map(fg -> Pair.of(fg.getFileGroupId(), getLatestBaseFile(fg)))
return fetchAllStoredFileGroups()
.filter(fg -> !isFileGroupReplaced(fg))
.map(fg -> Pair.of(fg.getFileGroupId(), getLatestBaseFile(fg)))
.filter(p -> p.getValue().isPresent())
.map(p -> addBootstrapBaseFileIfPresent(p.getKey(), p.getValue().get()));
}
@@ -880,6 +970,31 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV
.fromJavaOptional(fetchLatestFileSlices(partitionPath).filter(fs -> fs.getFileId().equals(fileId)).findFirst());
}
private boolean isFileGroupReplaced(String partitionPath, String fileId) {
return isFileGroupReplaced(new HoodieFileGroupId(partitionPath, fileId));
}
private boolean isFileGroupReplaced(HoodieFileGroup fileGroup) {
return isFileGroupReplaced(fileGroup.getFileGroupId());
}
private boolean isFileGroupReplaced(HoodieFileGroupId fileGroup) {
return getReplaceInstant(fileGroup).isPresent();
}
private boolean isFileGroupReplacedBeforeAny(HoodieFileGroupId fileGroupId, List<String> instants) {
return isFileGroupReplacedBeforeOrOn(fileGroupId, instants.stream().max(Comparator.naturalOrder()).get());
}
private boolean isFileGroupReplacedBeforeOrOn(HoodieFileGroupId fileGroupId, String instant) {
Option<HoodieInstant> hoodieInstantOption = getReplaceInstant(fileGroupId);
if (!hoodieInstantOption.isPresent()) {
return false;
}
return HoodieTimeline.compareTimestamps(instant, GREATER_THAN_OR_EQUALS, hoodieInstantOption.get().getTimestamp());
}
@Override
public Option<HoodieInstant> getLastInstant() {
return getTimeline().lastInstant();

View File

@@ -43,10 +43,13 @@ public class FileSystemViewStorageConfig extends DefaultHoodieConfig {
"hoodie.filesystem.view.spillable.compaction.mem.fraction";
public static final String FILESYSTEM_VIEW_BOOTSTRAP_BASE_FILE_FRACTION =
"hoodie.filesystem.view.spillable.bootstrap.base.file.mem.fraction";
public static final String FILESYSTEM_VIEW_REPLACED_MEM_FRACTION =
"hoodie.filesystem.view.spillable.replaced.mem.fraction";
private static final String ROCKSDB_BASE_PATH_PROP = "hoodie.filesystem.view.rocksdb.base.path";
public static final String FILESTYSTEM_REMOTE_TIMELINE_CLIENT_TIMEOUT_SECS =
"hoodie.filesystem.view.remote.timeout.secs";
public static final FileSystemViewStorageType DEFAULT_VIEW_STORAGE_TYPE = FileSystemViewStorageType.MEMORY;
public static final FileSystemViewStorageType DEFAULT_SECONDARY_VIEW_STORAGE_TYPE = FileSystemViewStorageType.MEMORY;
public static final String DEFAULT_ROCKSDB_BASE_PATH = "/tmp/hoodie_timeline_rocksdb";
@@ -58,6 +61,7 @@ public class FileSystemViewStorageConfig extends DefaultHoodieConfig {
public static final String DEFAULT_VIEW_SPILLABLE_DIR = "/tmp/view_map/";
private static final Double DEFAULT_MEM_FRACTION_FOR_PENDING_COMPACTION = 0.01;
private static final Double DEFAULT_MEM_FRACTION_FOR_EXTERNAL_DATA_FILE = 0.05;
private static final Double DEFAULT_MEM_FRACTION_FOR_REPLACED_FILEGROUPS = 0.01;
private static final Long DEFAULT_MAX_MEMORY_FOR_VIEW = 100 * 1024 * 1024L; // 100 MB
/**
@@ -116,6 +120,12 @@ public class FileSystemViewStorageConfig extends DefaultHoodieConfig {
return reservedForExternalDataFile;
}
public long getMaxMemoryForReplacedFileGroups() {
long totalMemory = Long.parseLong(props.getProperty(FILESYSTEM_VIEW_SPILLABLE_MEM));
return new Double(totalMemory * Double.parseDouble(props.getProperty(FILESYSTEM_VIEW_REPLACED_MEM_FRACTION)))
.longValue();
}
public String getBaseStoreDir() {
return props.getProperty(FILESYSTEM_VIEW_SPILLABLE_DIR);
}
@@ -233,6 +243,8 @@ public class FileSystemViewStorageConfig extends DefaultHoodieConfig {
FILESYSTEM_VIEW_PENDING_COMPACTION_MEM_FRACTION, DEFAULT_MEM_FRACTION_FOR_PENDING_COMPACTION.toString());
setDefaultOnCondition(props, !props.containsKey(FILESYSTEM_VIEW_BOOTSTRAP_BASE_FILE_FRACTION),
FILESYSTEM_VIEW_BOOTSTRAP_BASE_FILE_FRACTION, DEFAULT_MEM_FRACTION_FOR_EXTERNAL_DATA_FILE.toString());
setDefaultOnCondition(props, !props.containsKey(FILESYSTEM_VIEW_REPLACED_MEM_FRACTION),
FILESYSTEM_VIEW_REPLACED_MEM_FRACTION, DEFAULT_MEM_FRACTION_FOR_REPLACED_FILEGROUPS.toString());
setDefaultOnCondition(props, !props.containsKey(ROCKSDB_BASE_PATH_PROP), ROCKSDB_BASE_PATH_PROP,
DEFAULT_ROCKSDB_BASE_PATH);

View File

@@ -18,17 +18,17 @@
package org.apache.hudi.common.table.view;
import org.apache.hudi.common.model.CompactionOperation;
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.HoodieFileGroup;
import org.apache.hudi.common.model.HoodieFileGroupId;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.timeline.HoodieInstant;
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;
@@ -37,6 +37,7 @@ import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.stream.Collectors;
import java.util.stream.Stream;
@@ -64,6 +65,11 @@ public class HoodieTableFileSystemView extends IncrementalTimelineSyncFileSystem
*/
protected Map<HoodieFileGroupId, BootstrapBaseFileMapping> fgIdToBootstrapBaseFile;
/**
* Track replace time for replaced file groups.
*/
protected Map<HoodieFileGroupId, HoodieInstant> fgIdToReplaceInstants;
/**
* Flag to determine if closed.
*/
@@ -106,6 +112,7 @@ public class HoodieTableFileSystemView extends IncrementalTimelineSyncFileSystem
this.fgIdToPendingCompaction = null;
this.partitionToFileGroupsMap = null;
this.fgIdToBootstrapBaseFile = null;
this.fgIdToReplaceInstants = null;
}
protected Map<String, List<HoodieFileGroup>> createPartitionToFileGroups() {
@@ -122,6 +129,11 @@ public class HoodieTableFileSystemView extends IncrementalTimelineSyncFileSystem
return fileGroupIdBootstrapBaseFileMap;
}
protected Map<HoodieFileGroupId, HoodieInstant> createFileIdToReplaceInstantMap(final Map<HoodieFileGroupId, HoodieInstant> replacedFileGroups) {
Map<HoodieFileGroupId, HoodieInstant> replacedFileGroupsMap = new ConcurrentHashMap<>(replacedFileGroups);
return replacedFileGroupsMap;
}
/**
* Create a file system view, as of the given timeline, with the provided file statuses.
*/
@@ -261,6 +273,26 @@ public class HoodieTableFileSystemView extends IncrementalTimelineSyncFileSystem
return partitionToFileGroupsMap.values().stream().flatMap(Collection::stream);
}
@Override
protected void resetReplacedFileGroups(final Map<HoodieFileGroupId, HoodieInstant> replacedFileGroups) {
fgIdToReplaceInstants = createFileIdToReplaceInstantMap(replacedFileGroups);
}
@Override
protected void addReplacedFileGroups(final Map<HoodieFileGroupId, HoodieInstant> replacedFileGroups) {
fgIdToReplaceInstants.putAll(replacedFileGroups);
}
@Override
protected void removeReplacedFileIdsAtInstants(Set<String> instants) {
fgIdToReplaceInstants.entrySet().removeIf(entry -> instants.contains(entry.getValue().getTimestamp()));
}
@Override
protected Option<HoodieInstant> getReplaceInstant(final HoodieFileGroupId fileGroupId) {
return Option.ofNullable(fgIdToReplaceInstants.get(fileGroupId));
}
@Override
public void close() {
closed = true;
@@ -268,6 +300,7 @@ public class HoodieTableFileSystemView extends IncrementalTimelineSyncFileSystem
partitionToFileGroupsMap = null;
fgIdToPendingCompaction = null;
fgIdToBootstrapBaseFile = null;
fgIdToReplaceInstants = null;
}
@Override

View File

@@ -18,6 +18,8 @@
package org.apache.hudi.common.table.view;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.avro.model.HoodieCleanMetadata;
import org.apache.hudi.avro.model.HoodieCompactionPlan;
import org.apache.hudi.avro.model.HoodieRestoreMetadata;
@@ -28,7 +30,10 @@ import org.apache.hudi.common.model.FileSlice;
import org.apache.hudi.common.model.HoodieBaseFile;
import org.apache.hudi.common.model.HoodieCommitMetadata;
import org.apache.hudi.common.model.HoodieFileGroup;
import org.apache.hudi.common.model.HoodieFileGroupId;
import org.apache.hudi.common.model.HoodieLogFile;
import org.apache.hudi.common.model.HoodieReplaceCommitMetadata;
import org.apache.hudi.common.model.HoodieWriteStat;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.common.table.timeline.TimelineDiffHelper;
@@ -39,15 +44,13 @@ import org.apache.hudi.common.util.CompactionUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.exception.HoodieException;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import java.io.IOException;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.stream.Collectors;
/**
@@ -129,6 +132,8 @@ public abstract class IncrementalTimelineSyncFileSystemView extends AbstractTabl
addPendingCompactionInstant(timeline, instant);
} else if (instant.getAction().equals(HoodieTimeline.ROLLBACK_ACTION)) {
addRollbackInstant(timeline, instant);
} else if (instant.getAction().equals(HoodieTimeline.REPLACE_COMMIT_ACTION)) {
addReplaceInstant(timeline, instant);
}
} catch (IOException ioe) {
throw new HoodieException(ioe);
@@ -191,7 +196,14 @@ public abstract class IncrementalTimelineSyncFileSystemView extends AbstractTabl
LOG.info("Syncing committed instant (" + instant + ")");
HoodieCommitMetadata commitMetadata =
HoodieCommitMetadata.fromBytes(timeline.getInstantDetails(instant).get(), HoodieCommitMetadata.class);
commitMetadata.getPartitionToWriteStats().entrySet().stream().forEach(entry -> {
updatePartitionWriteFileGroups(commitMetadata.getPartitionToWriteStats(), timeline, instant);
LOG.info("Done Syncing committed instant (" + instant + ")");
}
private void updatePartitionWriteFileGroups(Map<String, List<HoodieWriteStat>> partitionToWriteStats,
HoodieTimeline timeline,
HoodieInstant instant) {
partitionToWriteStats.entrySet().stream().forEach(entry -> {
String partition = entry.getKey();
if (isPartitionAvailableInStore(partition)) {
LOG.info("Syncing partition (" + partition + ") of instant (" + instant + ")");
@@ -231,6 +243,13 @@ public abstract class IncrementalTimelineSyncFileSystemView extends AbstractTabl
removeFileSlicesForPartition(timeline, instant, e.getKey(),
e.getValue().stream().map(x -> x.getValue()).collect(Collectors.toList()));
});
if (metadata.getRestoreInstantInfo() != null) {
Set<String> rolledbackInstants = metadata.getRestoreInstantInfo().stream()
.filter(instantInfo -> HoodieTimeline.REPLACE_COMMIT_ACTION.equals(instantInfo.getAction()))
.map(instantInfo -> instantInfo.getCommitTime()).collect(Collectors.toSet());
removeReplacedFileIdsAtInstants(rolledbackInstants);
}
LOG.info("Done Syncing restore instant (" + instant + ")");
}
@@ -251,6 +270,28 @@ public abstract class IncrementalTimelineSyncFileSystemView extends AbstractTabl
LOG.info("Done Syncing rollback instant (" + instant + ")");
}
/**
* Add newly found REPLACE instant.
*
* @param timeline Hoodie Timeline
* @param instant REPLACE Instant
*/
private void addReplaceInstant(HoodieTimeline timeline, HoodieInstant instant) throws IOException {
LOG.info("Syncing replace instant (" + instant + ")");
HoodieReplaceCommitMetadata replaceMetadata =
HoodieReplaceCommitMetadata.fromBytes(timeline.getInstantDetails(instant).get(), HoodieReplaceCommitMetadata.class);
updatePartitionWriteFileGroups(replaceMetadata.getPartitionToWriteStats(), timeline, instant);
replaceMetadata.getPartitionToReplaceFileIds().entrySet().stream().forEach(entry -> {
String partition = entry.getKey();
Map<HoodieFileGroupId, HoodieInstant> replacedFileIds = entry.getValue().stream()
.collect(Collectors.toMap(replaceStat -> new HoodieFileGroupId(partition, replaceStat), replaceStat -> instant));
LOG.info("For partition (" + partition + ") of instant (" + instant + "), excluding " + replacedFileIds.size() + " file groups");
addReplacedFileGroups(replacedFileIds);
});
LOG.info("Done Syncing REPLACE instant (" + instant + ")");
}
/**
* Add newly found clean instant. Note that cleaner metadata (.clean.completed)
* contains only relative paths unlike clean plans (.clean.requested) which contains absolute paths.

View File

@@ -193,6 +193,11 @@ public class PriorityBasedFileSystemView implements SyncableFileSystemView, Seri
return execute(partitionPath, preferredView::getAllFileGroups, secondaryView::getAllFileGroups);
}
@Override
public Stream<HoodieFileGroup> getReplacedFileGroupsBeforeOrOn(String maxCommitTime, String partitionPath) {
return execute(maxCommitTime, partitionPath, preferredView::getReplacedFileGroupsBeforeOrOn, secondaryView::getReplacedFileGroupsBeforeOrOn);
}
@Override
public Stream<Pair<String, CompactionOperation>> getPendingCompactionOperations() {
return execute(preferredView::getPendingCompactionOperations, secondaryView::getPendingCompactionOperations);

View File

@@ -18,6 +18,11 @@
package org.apache.hudi.common.table.view;
import com.fasterxml.jackson.core.type.TypeReference;
import com.fasterxml.jackson.databind.ObjectMapper;
import org.apache.http.client.fluent.Request;
import org.apache.http.client.fluent.Response;
import org.apache.http.client.utils.URIBuilder;
import org.apache.hudi.common.model.CompactionOperation;
import org.apache.hudi.common.model.FileSlice;
import org.apache.hudi.common.model.HoodieBaseFile;
@@ -36,12 +41,6 @@ import org.apache.hudi.common.util.StringUtils;
import org.apache.hudi.common.util.ValidationUtils;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.exception.HoodieRemoteException;
import com.fasterxml.jackson.core.type.TypeReference;
import com.fasterxml.jackson.databind.ObjectMapper;
import org.apache.http.client.fluent.Request;
import org.apache.http.client.fluent.Response;
import org.apache.http.client.utils.URIBuilder;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
@@ -87,6 +86,9 @@ public class RemoteHoodieTableFileSystemView implements SyncableFileSystemView,
public static final String ALL_FILEGROUPS_FOR_PARTITION_URL =
String.format("%s/%s", BASE_URL, "filegroups/all/partition/");
public static final String ALL_REPLACED_FILEGROUPS_BEFORE_OR_ON =
String.format("%s/%s", BASE_URL, "filegroups/replaced/beforeoron/");
public static final String LAST_INSTANT = String.format("%s/%s", BASE_URL, "timeline/instant/last");
public static final String LAST_INSTANTS = String.format("%s/%s", BASE_URL, "timeline/instants/last");
@@ -361,6 +363,18 @@ public class RemoteHoodieTableFileSystemView implements SyncableFileSystemView,
}
}
@Override
public Stream<HoodieFileGroup> getReplacedFileGroupsBeforeOrOn(String maxCommitTime, String partitionPath) {
Map<String, String> paramsMap = getParamsWithAdditionalParam(partitionPath, MAX_INSTANT_PARAM, maxCommitTime);
try {
List<FileGroupDTO> fileGroups = executeRequest(ALL_REPLACED_FILEGROUPS_BEFORE_OR_ON, paramsMap,
new TypeReference<List<FileGroupDTO>>() {}, RequestMethod.GET);
return fileGroups.stream().map(dto -> FileGroupDTO.toFileGroup(dto, metaClient));
} catch (IOException e) {
throw new HoodieRemoteException(e);
}
}
public boolean refresh() {
Map<String, String> paramsMap = getParams();
try {

View File

@@ -18,23 +18,23 @@
package org.apache.hudi.common.table.view;
import org.apache.hudi.common.model.CompactionOperation;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
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.HoodieBaseFile;
import org.apache.hudi.common.model.HoodieFileGroup;
import org.apache.hudi.common.model.HoodieFileGroupId;
import org.apache.hudi.common.model.HoodieLogFile;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.RocksDBSchemaHelper;
import org.apache.hudi.common.util.ValidationUtils;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.common.util.collection.RocksDBDAO;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
@@ -43,6 +43,7 @@ import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Set;
import java.util.stream.Collectors;
import java.util.stream.Stream;
@@ -371,6 +372,59 @@ public class RocksDbBasedFileSystemView extends IncrementalTimelineSyncFileSyste
schemaHelper.getPrefixForSliceViewByPartitionFile(partitionPath, fileId)).map(Pair::getValue)).findFirst());
}
@Override
protected void resetReplacedFileGroups(final Map<HoodieFileGroupId, HoodieInstant> replacedFileGroups) {
LOG.info("Resetting replacedFileGroups to ROCKSDB based file-system view at "
+ config.getRocksdbBasePath() + ", Total file-groups=" + replacedFileGroups.size());
// Delete all replaced file groups
rocksDB.prefixDelete(schemaHelper.getColFamilyForReplacedFileGroups(), "part=");
// Now add new entries
addReplacedFileGroups(replacedFileGroups);
LOG.info("Resetting replacedFileGroups to ROCKSDB based file-system view complete");
}
@Override
protected void addReplacedFileGroups(final Map<HoodieFileGroupId, HoodieInstant> replacedFileGroups) {
Map<String, List<Map.Entry<HoodieFileGroupId, HoodieInstant>>> partitionToReplacedFileGroups =
replacedFileGroups.entrySet().stream().collect(Collectors.groupingBy(e -> e.getKey().getPartitionPath()));
partitionToReplacedFileGroups.entrySet().stream().forEach(partitionToReplacedFileGroupsEntry -> {
String partitionPath = partitionToReplacedFileGroupsEntry.getKey();
List<Map.Entry<HoodieFileGroupId, HoodieInstant>> replacedFileGroupsInPartition = partitionToReplacedFileGroupsEntry.getValue();
// Now add them
rocksDB.writeBatch(batch ->
replacedFileGroupsInPartition.stream().forEach(fgToReplacedInstant -> {
rocksDB.putInBatch(batch, schemaHelper.getColFamilyForReplacedFileGroups(),
schemaHelper.getKeyForReplacedFileGroup(fgToReplacedInstant.getKey()), fgToReplacedInstant.getValue());
})
);
LOG.info("Finished adding replaced file groups to partition (" + partitionPath + ") to ROCKSDB based view at "
+ config.getRocksdbBasePath() + ", Total file-groups=" + partitionToReplacedFileGroupsEntry.getValue().size());
});
}
@Override
protected void removeReplacedFileIdsAtInstants(Set<String> instants) {
//TODO can we make this more efficient by storing reverse mapping (Instant -> FileGroupId) as well?
Stream<String> keysToDelete = rocksDB.<HoodieInstant>prefixSearch(schemaHelper.getColFamilyForReplacedFileGroups(), "")
.filter(entry -> instants.contains(entry.getValue().getTimestamp()))
.map(Pair::getKey);
rocksDB.writeBatch(batch ->
keysToDelete.forEach(key -> rocksDB.deleteInBatch(batch, schemaHelper.getColFamilyForReplacedFileGroups(), key))
);
}
@Override
protected Option<HoodieInstant> getReplaceInstant(final HoodieFileGroupId fileGroupId) {
String lookupKey = schemaHelper.getKeyForReplacedFileGroup(fileGroupId);
HoodieInstant replacedInstant =
rocksDB.get(schemaHelper.getColFamilyForReplacedFileGroups(), lookupKey);
return Option.ofNullable(replacedInstant);
}
private Stream<HoodieFileGroup> getFileGroups(Stream<FileSlice> sliceStream) {
return sliceStream.map(s -> Pair.of(Pair.of(s.getPartitionPath(), s.getFileId()), s))
.collect(Collectors.groupingBy(Pair::getKey)).entrySet().stream().map(slicePair -> {

View File

@@ -18,24 +18,28 @@
package org.apache.hudi.common.table.view;
import java.io.File;
import java.io.IOException;
import java.util.List;
import java.util.Map;
import java.util.stream.Stream;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hudi.common.model.CompactionOperation;
import org.apache.hudi.common.model.BootstrapBaseFileMapping;
import org.apache.hudi.common.model.CompactionOperation;
import org.apache.hudi.common.model.HoodieFileGroup;
import org.apache.hudi.common.model.HoodieFileGroupId;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.common.util.DefaultSizeEstimator;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.collection.ExternalSpillableMap;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import java.io.File;
import java.io.IOException;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.stream.Stream;
/**
* Table FileSystemView implementation where view is stored in spillable disk using fixed memory.
*/
@@ -46,6 +50,7 @@ public class SpillableMapBasedFileSystemView extends HoodieTableFileSystemView {
private final long maxMemoryForFileGroupMap;
private final long maxMemoryForPendingCompaction;
private final long maxMemoryForBootstrapBaseFile;
private final long maxMemoryForReplaceFileGroups;
private final String baseStoreDir;
public SpillableMapBasedFileSystemView(HoodieTableMetaClient metaClient, HoodieTimeline visibleActiveTimeline,
@@ -54,6 +59,7 @@ public class SpillableMapBasedFileSystemView extends HoodieTableFileSystemView {
this.maxMemoryForFileGroupMap = config.getMaxMemoryForFileGroupMap();
this.maxMemoryForPendingCompaction = config.getMaxMemoryForPendingCompaction();
this.maxMemoryForBootstrapBaseFile = config.getMaxMemoryForBootstrapBaseFile();
this.maxMemoryForReplaceFileGroups = config.getMaxMemoryForReplacedFileGroups();
this.baseStoreDir = config.getBaseStoreDir();
init(metaClient, visibleActiveTimeline);
}
@@ -109,6 +115,21 @@ public class SpillableMapBasedFileSystemView extends HoodieTableFileSystemView {
}
}
@Override
protected Map<HoodieFileGroupId, HoodieInstant> createFileIdToReplaceInstantMap(final Map<HoodieFileGroupId, HoodieInstant> replacedFileGroups) {
try {
LOG.info("Creating file group id to replace instant map using external spillable Map. Max Mem=" + maxMemoryForReplaceFileGroups
+ ", BaseDir=" + baseStoreDir);
new File(baseStoreDir).mkdirs();
Map<HoodieFileGroupId, HoodieInstant> pendingMap = new ExternalSpillableMap<>(
maxMemoryForReplaceFileGroups, baseStoreDir, new DefaultSizeEstimator(), new DefaultSizeEstimator<>());
pendingMap.putAll(replacedFileGroups);
return pendingMap;
} catch (IOException e) {
throw new RuntimeException(e);
}
}
@Override
public Stream<HoodieFileGroup> getAllFileGroups() {
return ((ExternalSpillableMap) partitionToFileGroupsMap).valueStream()
@@ -129,4 +150,18 @@ public class SpillableMapBasedFileSystemView extends HoodieTableFileSystemView {
public Stream<HoodieFileGroup> fetchAllStoredFileGroups() {
return ((ExternalSpillableMap) partitionToFileGroupsMap).valueStream().flatMap(fg -> ((List<HoodieFileGroup>) fg).stream());
}
@Override
protected void removeReplacedFileIdsAtInstants(Set<String> instants) {
//TODO should we make this more efficient by having reverse mapping of instant to file group id?
Stream<HoodieFileGroupId> fileIdsToRemove = fgIdToReplaceInstants.entrySet().stream().map(entry -> {
if (instants.contains(entry.getValue().getTimestamp())) {
return Option.of(entry.getKey());
} else {
return Option.ofNullable((HoodieFileGroupId) null);
}
}).filter(Option::isPresent).map(Option::get);
fileIdsToRemove.forEach(fileGroupId -> fgIdToReplaceInstants.remove(fileGroupId));
}
}

View File

@@ -164,4 +164,9 @@ public interface TableFileSystemView {
* Timeline corresponding to the view.
*/
HoodieTimeline getTimeline();
/**
* Stream all the replaced file groups before maxCommitTime.
*/
Stream<HoodieFileGroup> getReplacedFileGroupsBeforeOrOn(String maxCommitTime, String partitionPath);
}

View File

@@ -0,0 +1,96 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common.util;
import org.apache.hudi.common.model.HoodieCommitMetadata;
import org.apache.hudi.common.model.HoodieReplaceCommitMetadata;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.model.HoodieWriteStat;
import org.apache.hudi.common.model.WriteOperationType;
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.exception.HoodieException;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import java.util.List;
import java.util.Map;
/**
* Helper class to generate commit metadata.
*/
public class CommitUtils {
private static final Logger LOG = LogManager.getLogger(CommitUtils.class);
/**
* Gets the commit action type for given table type.
*/
public static String getCommitActionType(HoodieTableType tableType) {
switch (tableType) {
case COPY_ON_WRITE:
return HoodieActiveTimeline.COMMIT_ACTION;
case MERGE_ON_READ:
return HoodieActiveTimeline.DELTA_COMMIT_ACTION;
default:
throw new HoodieException("Could not commit on unknown table type " + tableType);
}
}
public static HoodieCommitMetadata buildMetadata(List<HoodieWriteStat> writeStats,
Map<String, List<String>> partitionToReplaceFileIds,
Option<Map<String, String>> extraMetadata,
WriteOperationType operationType,
String schemaToStoreInCommit,
String commitActionType) {
HoodieCommitMetadata commitMetadata = buildMetadataFromStats(writeStats, partitionToReplaceFileIds, commitActionType, operationType);
// add in extra metadata
if (extraMetadata.isPresent()) {
extraMetadata.get().forEach(commitMetadata::addMetadata);
}
commitMetadata.addMetadata(HoodieCommitMetadata.SCHEMA_KEY, schemaToStoreInCommit);
commitMetadata.setOperationType(operationType);
return commitMetadata;
}
private static HoodieCommitMetadata buildMetadataFromStats(List<HoodieWriteStat> writeStats,
Map<String, List<String>> partitionToReplaceFileIds,
String commitActionType,
WriteOperationType operationType) {
final HoodieCommitMetadata commitMetadata;
if (commitActionType == HoodieTimeline.REPLACE_COMMIT_ACTION) {
HoodieReplaceCommitMetadata replaceMetadata = new HoodieReplaceCommitMetadata();
replaceMetadata.setPartitionToReplaceFileIds(partitionToReplaceFileIds);
commitMetadata = replaceMetadata;
} else {
commitMetadata = new HoodieCommitMetadata();
}
for (HoodieWriteStat writeStat : writeStats) {
String partition = writeStat.getPartitionPath();
commitMetadata.addWriteStat(partition, writeStat);
}
LOG.info("Creating metadata for " + operationType + " numWriteStats:" + writeStats.size()
+ "numReplaceFileIds:" + partitionToReplaceFileIds.values().stream().mapToInt(e -> e.size()).sum());
return commitMetadata;
}
}

View File

@@ -47,17 +47,19 @@ public class RocksDBSchemaHelper {
private final String colFamilyForPendingCompaction;
private final String colFamilyForBootstrapBaseFile;
private final String colFamilyForStoredPartitions;
private final String colFamilyForReplacedFileGroups;
public RocksDBSchemaHelper(HoodieTableMetaClient metaClient) {
this.colFamilyForBootstrapBaseFile = "hudi_bootstrap_basefile_" + metaClient.getBasePath().replace("/", "_");
this.colFamilyForPendingCompaction = "hudi_pending_compaction_" + metaClient.getBasePath().replace("/", "_");
this.colFamilyForStoredPartitions = "hudi_partitions_" + metaClient.getBasePath().replace("/", "_");
this.colFamilyForView = "hudi_view_" + metaClient.getBasePath().replace("/", "_");
this.colFamilyForReplacedFileGroups = "hudi_replaced_fg" + metaClient.getBasePath().replace("/", "_");
}
public List<String> getAllColumnFamilies() {
return Arrays.asList(getColFamilyForView(), getColFamilyForPendingCompaction(), getColFamilyForBootstrapBaseFile(),
getColFamilyForStoredPartitions());
getColFamilyForStoredPartitions(), getColFamilyForReplacedFileGroups());
}
public String getKeyForPartitionLookup(String partition) {
@@ -72,6 +74,10 @@ public class RocksDBSchemaHelper {
return getPartitionFileIdBasedLookup(fgId);
}
public String getKeyForReplacedFileGroup(HoodieFileGroupId fgId) {
return getPartitionFileIdBasedLookup(fgId);
}
public String getKeyForSliceView(HoodieFileGroup fileGroup, FileSlice slice) {
return getKeyForSliceView(fileGroup.getPartitionPath(), fileGroup.getFileGroupId().getFileId(),
slice.getBaseInstantTime());
@@ -125,4 +131,8 @@ public class RocksDBSchemaHelper {
public String getColFamilyForStoredPartitions() {
return colFamilyForStoredPartitions;
}
public String getColFamilyForReplacedFileGroups() {
return colFamilyForReplacedFileGroups;
}
}