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

@@ -73,8 +73,10 @@
<import>${basedir}/src/main/avro/HoodieCleanPartitionMetadata.avsc</import>
<import>${basedir}/src/main/avro/HoodieCleanMetadata.avsc</import>
<import>${basedir}/src/main/avro/HoodieCleanerPlan.avsc</import>
<import>${basedir}/src/main/avro/HoodieInstantInfo.avsc</import>
<import>${basedir}/src/main/avro/HoodieRollbackMetadata.avsc</import>
<import>${basedir}/src/main/avro/HoodieRestoreMetadata.avsc</import>
<import>${basedir}/src/main/avro/HoodieReplaceCommitMetadata.avsc</import>
<import>${basedir}/src/main/avro/HoodieArchivedMetaEntry.avsc</import>
<import>${basedir}/src/main/avro/HoodiePath.avsc</import>
<import>${basedir}/src/main/avro/HoodieFSPermission.avsc</import>

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;
}
}

View File

@@ -135,7 +135,7 @@ public class TestTimelineUtils extends HoodieCommonTestHarness {
String ts = i + "";
HoodieInstant instant = new HoodieInstant(true, HoodieTimeline.RESTORE_ACTION, ts);
activeTimeline.createNewInstant(instant);
activeTimeline.saveAsComplete(instant, Option.of(getRestoreMetadata(basePath, ts, ts, 2)));
activeTimeline.saveAsComplete(instant, Option.of(getRestoreMetadata(basePath, ts, ts, 2, HoodieTimeline.COMMIT_ACTION)));
}
metaClient.reloadActiveTimeline();
@@ -184,25 +184,23 @@ public class TestTimelineUtils extends HoodieCommonTestHarness {
assertEquals(extraMetadataValue1, extraMetadataEntries.get("1").get());
}
private byte[] getRestoreMetadata(String basePath, String partition, String commitTs, int count) throws IOException {
HoodieRestoreMetadata metadata = new HoodieRestoreMetadata();
private byte[] getRestoreMetadata(String basePath, String partition, String commitTs, int count, String actionType) throws IOException {
List<HoodieRollbackMetadata> rollbackM = new ArrayList<>();
rollbackM.add(getRollbackMetadataInstance(basePath, partition, commitTs, count));
metadata.setHoodieRestoreMetadata(CollectionUtils.createImmutableMap(commitTs, rollbackM));
List<String> rollbackInstants = new ArrayList<>();
rollbackInstants.add(commitTs);
metadata.setInstantsToRollback(rollbackInstants);
metadata.setStartRestoreTime(commitTs);
rollbackM.add(getRollbackMetadataInstance(basePath, partition, commitTs, count, actionType));
List<HoodieInstant> rollbackInstants = new ArrayList<>();
rollbackInstants.add(new HoodieInstant(false, commitTs, actionType));
HoodieRestoreMetadata metadata = TimelineMetadataUtils.convertRestoreMetadata(commitTs, 200, rollbackInstants,
CollectionUtils.createImmutableMap(commitTs, rollbackM));
return TimelineMetadataUtils.serializeRestoreMetadata(metadata).get();
}
private HoodieRollbackMetadata getRollbackMetadataInstance(String basePath, String partition, String commitTs, int count) {
private HoodieRollbackMetadata getRollbackMetadataInstance(String basePath, String partition, String commitTs, int count, String actionType) {
List<String> deletedFiles = new ArrayList<>();
for (int i = 1; i <= count; i++) {
deletedFiles.add("file-" + i);
}
List<String> rollbacks = new ArrayList<>();
rollbacks.add(commitTs);
List<HoodieInstant> rollbacks = new ArrayList<>();
rollbacks.add(new HoodieInstant(false, actionType, commitTs));
HoodieRollbackStat rollbackStat = new HoodieRollbackStat(partition, deletedFiles, Collections.emptyList(), Collections.emptyMap());
List<HoodieRollbackStat> rollbackStats = new ArrayList<>();

View File

@@ -199,10 +199,10 @@ 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));
CollectionUtils.createSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION, HoodieTimeline.REPLACE_COMMIT_ACTION));
checkTimeline.accept(timeline.getCommitsAndCompactionTimeline(),
CollectionUtils.createSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION, HoodieTimeline.COMPACTION_ACTION));
checkTimeline.accept(timeline.getCommitTimeline(), Collections.singleton(HoodieTimeline.COMMIT_ACTION));
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));
@@ -210,7 +210,7 @@ public class TestHoodieActiveTimeline extends HoodieCommonTestHarness {
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.CLEAN_ACTION, HoodieTimeline.COMPACTION_ACTION, HoodieTimeline.REPLACE_COMMIT_ACTION,
HoodieTimeline.SAVEPOINT_ACTION, HoodieTimeline.ROLLBACK_ACTION));
// Get some random Instants
@@ -404,6 +404,27 @@ public class TestHoodieActiveTimeline extends HoodieCommonTestHarness {
.forEach(i -> assertFalse(t2.containsInstant(i)));
}
@Test
public void testReplaceActionsTimeline() {
int instantTime = 1;
List<HoodieInstant> allInstants = new ArrayList<>();
HoodieInstant instant = new HoodieInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, String.format("%03d", instantTime++));
allInstants.add(instant);
instant = new HoodieInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, String.format("%03d", instantTime++));
allInstants.add(instant);
instant = new HoodieInstant(State.COMPLETED, HoodieTimeline.REPLACE_COMMIT_ACTION, String.format("%03d", instantTime++));
allInstants.add(instant);
timeline = new HoodieActiveTimeline(metaClient);
timeline.setInstants(allInstants);
List<HoodieInstant> validReplaceInstants =
timeline.getCompletedReplaceTimeline().getInstants().collect(Collectors.toList());
assertEquals(1, validReplaceInstants.size());
assertEquals(instant.getTimestamp(), validReplaceInstants.get(0).getTimestamp());
assertEquals(HoodieTimeline.REPLACE_COMMIT_ACTION, validReplaceInstants.get(0).getAction());
}
/**
* Returns an exhaustive list of all possible HoodieInstant.
* @return list of HoodieInstant

View File

@@ -18,6 +18,9 @@
package org.apache.hudi.common.table.view;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.permission.FsAction;
import org.apache.hudi.avro.model.HoodieCompactionPlan;
import org.apache.hudi.avro.model.HoodieFSPermission;
import org.apache.hudi.avro.model.HoodieFileStatus;
@@ -31,10 +34,12 @@ import org.apache.hudi.common.model.BootstrapFileMapping;
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.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.HoodieTableType;
import org.apache.hudi.common.model.WriteOperationType;
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.timeline.HoodieInstant.State;
@@ -44,13 +49,10 @@ import org.apache.hudi.common.table.view.TableFileSystemView.BaseFileOnlyView;
import org.apache.hudi.common.table.view.TableFileSystemView.SliceView;
import org.apache.hudi.common.testutils.HoodieCommonTestHarness;
import org.apache.hudi.common.testutils.HoodieTestUtils;
import org.apache.hudi.common.util.CommitUtils;
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.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.permission.FsAction;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.junit.jupiter.api.BeforeEach;
@@ -61,11 +63,13 @@ import org.junit.jupiter.params.provider.MethodSource;
import java.io.File;
import java.io.IOException;
import java.nio.charset.StandardCharsets;
import java.nio.file.Paths;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.Date;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
@@ -1273,6 +1277,153 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
}
}
@Test
public void testReplaceWithTimeTravel() throws IOException {
String partitionPath1 = "2020/06/27";
new File(basePath + "/" + partitionPath1).mkdirs();
// create 2 fileId in partition1 - fileId1 is replaced later on.
String fileId1 = UUID.randomUUID().toString();
String fileId2 = UUID.randomUUID().toString();
assertFalse(roView.getLatestBaseFiles(partitionPath1)
.anyMatch(dfile -> dfile.getFileId().equals(fileId1) || dfile.getFileId().equals(fileId2)),
"No commit, should not find any data file");
// Only one commit
String commitTime1 = "1";
String fileName1 = FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId1);
String fileName2 = FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId2);
new File(basePath + "/" + partitionPath1 + "/" + fileName1).createNewFile();
new File(basePath + "/" + partitionPath1 + "/" + fileName2).createNewFile();
HoodieActiveTimeline commitTimeline = metaClient.getActiveTimeline();
HoodieInstant instant1 = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, commitTime1);
saveAsComplete(commitTimeline, instant1, Option.empty());
refreshFsView();
assertEquals(1, roView.getLatestBaseFiles(partitionPath1)
.filter(dfile -> dfile.getFileId().equals(fileId1)).count());
assertEquals(1, roView.getLatestBaseFiles(partitionPath1)
.filter(dfile -> dfile.getFileId().equals(fileId2)).count());
// create commit2 - fileId1 is replaced. new file groups fileId3,fileId4 are created.
String fileId3 = UUID.randomUUID().toString();
String fileId4 = UUID.randomUUID().toString();
String fileName3 = FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId3);
String fileName4 = FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId4);
new File(basePath + "/" + partitionPath1 + "/" + fileName3).createNewFile();
new File(basePath + "/" + partitionPath1 + "/" + fileName4).createNewFile();
String commitTime2 = "2";
Map<String, List<String>> partitionToReplaceFileIds = new HashMap<>();
List<String> replacedFileIds = new ArrayList<>();
replacedFileIds.add(fileId1);
partitionToReplaceFileIds.put(partitionPath1, replacedFileIds);
HoodieCommitMetadata commitMetadata =
CommitUtils.buildMetadata(Collections.emptyList(), partitionToReplaceFileIds, Option.empty(), WriteOperationType.INSERT_OVERWRITE, "", HoodieTimeline.REPLACE_COMMIT_ACTION);
commitTimeline = metaClient.getActiveTimeline();
HoodieInstant instant2 = new HoodieInstant(true, HoodieTimeline.REPLACE_COMMIT_ACTION, commitTime2);
saveAsComplete(commitTimeline, instant2, Option.of(commitMetadata.toJsonString().getBytes(StandardCharsets.UTF_8)));
//make sure view doesnt include fileId1
refreshFsView();
assertEquals(0, roView.getLatestBaseFiles(partitionPath1)
.filter(dfile -> dfile.getFileId().equals(fileId1)).count());
assertEquals(1, roView.getLatestBaseFiles(partitionPath1)
.filter(dfile -> dfile.getFileId().equals(fileId2)).count());
assertEquals(1, roView.getLatestBaseFiles(partitionPath1)
.filter(dfile -> dfile.getFileId().equals(fileId3)).count());
assertEquals(1, roView.getLatestBaseFiles(partitionPath1)
.filter(dfile -> dfile.getFileId().equals(fileId4)).count());
//exclude commit 2 and make sure fileId1 shows up in view.
SyncableFileSystemView filteredView = getFileSystemView(metaClient.getActiveTimeline().findInstantsBefore("2"), false);
assertEquals(1, filteredView.getLatestBaseFiles(partitionPath1)
.filter(dfile -> dfile.getFileId().equals(fileId1)).count());
assertEquals(1, filteredView.getLatestBaseFiles(partitionPath1)
.filter(dfile -> dfile.getFileId().equals(fileId2)).count());
assertEquals(1, filteredView.getLatestBaseFiles(partitionPath1)
.filter(dfile -> dfile.getFileId().equals(fileId3)).count());
assertEquals(1, filteredView.getLatestBaseFiles(partitionPath1)
.filter(dfile -> dfile.getFileId().equals(fileId4)).count());
// ensure replacedFileGroupsBefore works with all instants
List<HoodieFileGroup> replacedOnInstant1 = fsView.getReplacedFileGroupsBeforeOrOn("1", partitionPath1).collect(Collectors.toList());
assertEquals(0, replacedOnInstant1.size());
List<HoodieFileGroup> allReplaced = fsView.getReplacedFileGroupsBeforeOrOn("2", partitionPath1).collect(Collectors.toList());
assertEquals(1, allReplaced.size());
assertEquals(fileId1, allReplaced.get(0).getFileGroupId().getFileId());
}
@Test
public void testReplaceFileIdIsExcludedInView() throws IOException {
String partitionPath1 = "2020/06/27";
String partitionPath2 = "2020/07/14";
new File(basePath + "/" + partitionPath1).mkdirs();
new File(basePath + "/" + partitionPath2).mkdirs();
// create 2 fileId in partition1 - fileId1 is replaced later on.
String fileId1 = UUID.randomUUID().toString();
String fileId2 = UUID.randomUUID().toString();
// create 2 fileId in partition2 - fileId3, fileId4 is replaced later on.
String fileId3 = UUID.randomUUID().toString();
String fileId4 = UUID.randomUUID().toString();
assertFalse(roView.getLatestBaseFiles(partitionPath1)
.anyMatch(dfile -> dfile.getFileId().equals(fileId1) || dfile.getFileId().equals(fileId2)),
"No commit, should not find any data file");
assertFalse(roView.getLatestBaseFiles(partitionPath2)
.anyMatch(dfile -> dfile.getFileId().equals(fileId3) || dfile.getFileId().equals(fileId4)),
"No commit, should not find any data file");
// Only one commit
String commitTime1 = "1";
String fileName1 = FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId1);
String fileName2 = FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId2);
String fileName3 = FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId3);
String fileName4 = FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId4);
new File(basePath + "/" + partitionPath1 + "/" + fileName1).createNewFile();
new File(basePath + "/" + partitionPath1 + "/" + fileName2).createNewFile();
new File(basePath + "/" + partitionPath2 + "/" + fileName3).createNewFile();
new File(basePath + "/" + partitionPath2 + "/" + fileName4).createNewFile();
Map<String, List<String>> partitionToReplaceFileIds = new HashMap<>();
List<String> replacedFileIdsP1 = new ArrayList<>();
replacedFileIdsP1.add(fileId1);
partitionToReplaceFileIds.put(partitionPath1, replacedFileIdsP1);
List<String> replacedFileIdsP2 = new ArrayList<>();
replacedFileIdsP2.add(fileId3);
replacedFileIdsP2.add(fileId4);
partitionToReplaceFileIds.put(partitionPath2, replacedFileIdsP2);
HoodieCommitMetadata commitMetadata =
CommitUtils.buildMetadata(Collections.emptyList(), partitionToReplaceFileIds, Option.empty(), WriteOperationType.INSERT_OVERWRITE, "", HoodieTimeline.REPLACE_COMMIT_ACTION);
HoodieActiveTimeline commitTimeline = metaClient.getActiveTimeline();
HoodieInstant instant1 = new HoodieInstant(true, HoodieTimeline.REPLACE_COMMIT_ACTION, commitTime1);
saveAsComplete(commitTimeline, instant1, Option.of(commitMetadata.toJsonString().getBytes(StandardCharsets.UTF_8)));
refreshFsView();
assertEquals(0, roView.getLatestBaseFiles(partitionPath1)
.filter(dfile -> dfile.getFileId().equals(fileId1)).count());
assertEquals(fileName2, roView.getLatestBaseFiles(partitionPath1)
.filter(dfile -> dfile.getFileId().equals(fileId2)).findFirst().get().getFileName());
assertEquals(0, roView.getLatestBaseFiles(partitionPath2)
.filter(dfile -> dfile.getFileId().equals(fileId3)).count());
assertEquals(0, roView.getLatestBaseFiles(partitionPath2)
.filter(dfile -> dfile.getFileId().equals(fileId4)).count());
// ensure replacedFileGroupsBefore works with all instants
List<HoodieFileGroup> replacedOnInstant1 = fsView.getReplacedFileGroupsBeforeOrOn("0", partitionPath1).collect(Collectors.toList());
assertEquals(0, replacedOnInstant1.size());
List<HoodieFileGroup> allReplaced = fsView.getReplacedFileGroupsBeforeOrOn("2", partitionPath1).collect(Collectors.toList());
allReplaced.addAll(fsView.getReplacedFileGroupsBeforeOrOn("2", partitionPath2).collect(Collectors.toList()));
assertEquals(3, allReplaced.size());
Set<String> allReplacedFileIds = allReplaced.stream().map(fg -> fg.getFileGroupId().getFileId()).collect(Collectors.toSet());
Set<String> actualReplacedFileIds = Stream.of(fileId1, fileId3, fileId4).collect(Collectors.toSet());
assertEquals(actualReplacedFileIds, allReplacedFileIds);
}
@Override
protected HoodieTableType getTableType() {
return HoodieTableType.MERGE_ON_READ;

View File

@@ -32,6 +32,7 @@ import org.apache.hudi.common.model.HoodieCleaningPolicy;
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.HoodieReplaceCommitMetadata;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.model.HoodieWriteStat;
import org.apache.hudi.common.table.HoodieTableMetaClient;
@@ -49,6 +50,7 @@ import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.exception.HoodieException;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.junit.jupiter.api.BeforeEach;
@@ -62,6 +64,7 @@ import java.nio.file.Paths;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.Comparator;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
@@ -81,12 +84,13 @@ import static org.junit.jupiter.api.Assertions.assertTrue;
public class TestIncrementalFSViewSync extends HoodieCommonTestHarness {
private static final Logger LOG = LogManager.getLogger(TestIncrementalFSViewSync.class);
private static final int NUM_FILE_IDS_PER_PARTITION = 10;
private static String TEST_WRITE_TOKEN = "1-0-1";
private final List<String> partitions = Arrays.asList("2018/01/01", "2018/01/02", "2019/03/01");
private final List<String> fileIdsPerPartition =
IntStream.range(0, 10).mapToObj(x -> UUID.randomUUID().toString()).collect(Collectors.toList());
IntStream.range(0, NUM_FILE_IDS_PER_PARTITION).mapToObj(x -> UUID.randomUUID().toString()).collect(Collectors.toList());
@BeforeEach
public void init() throws IOException {
@@ -143,13 +147,13 @@ public class TestIncrementalFSViewSync extends HoodieCommonTestHarness {
{
put("11", Arrays.asList("12", "13", "15"));
}
}, instantsToFiles, Collections.singletonList("11"));
}, instantsToFiles, Collections.singletonList("11"), 0, 0);
// Add one more ingestion instant. This should be 2nd slice now
instantsToFiles.putAll(testMultipleWriteSteps(view, Collections.singletonList("22"), true, "19", 2));
// Restore last ingestion
testRestore(view, Collections.singletonList("23"), true, new HashMap<>(), Collections.singletonList("22"), "24", false);
testRestore(view, Collections.singletonList("23"), new HashMap<>(), Collections.singletonList(getHoodieCommitInstant("22", true)), "24", false);
// Run one more ingestion. THis is still 2nd slice
instantsToFiles.putAll(testMultipleWriteSteps(view, Collections.singletonList("24"), true, "19", 2));
@@ -188,14 +192,117 @@ public class TestIncrementalFSViewSync extends HoodieCommonTestHarness {
Map<String, List<String>> instantsToFiles = testMultipleWriteSteps(view, Arrays.asList("12", "13", "14"));
// restore instants in reverse order till we rollback all
testRestore(view, Arrays.asList("15", "16", "17"), false, instantsToFiles, Arrays.asList("14", "13", "12"), "17",
true);
testRestore(view, Arrays.asList("15", "16", "17"), instantsToFiles,
Arrays.asList(getHoodieCommitInstant("14", false), getHoodieCommitInstant("13", false), getHoodieCommitInstant("12", false)),
"17", true);
// Add 5 non-empty ingestions back-to-back
instantsToFiles = testMultipleWriteSteps(view, Arrays.asList("18", "19", "20"));
// Clean instants.
testCleans(view, Arrays.asList("21", "22"), instantsToFiles, Arrays.asList("18", "19"));
testCleans(view, Arrays.asList("21", "22"), instantsToFiles, Arrays.asList("18", "19"), 0, 0);
}
@Test
public void testReplaceCommits() throws IOException {
SyncableFileSystemView view = getFileSystemView(metaClient);
// Add an empty ingestion
String firstEmptyInstantTs = "11";
HoodieCommitMetadata metadata = new HoodieCommitMetadata();
metaClient.getActiveTimeline().createNewInstant(
new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, firstEmptyInstantTs));
metaClient.getActiveTimeline().saveAsComplete(
new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, firstEmptyInstantTs),
Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8)));
view.sync();
assertTrue(view.getLastInstant().isPresent());
assertEquals("11", view.getLastInstant().get().getTimestamp());
assertEquals(State.COMPLETED, view.getLastInstant().get().getState());
assertEquals(HoodieTimeline.COMMIT_ACTION, view.getLastInstant().get().getAction());
partitions.forEach(p -> assertEquals(0, view.getLatestFileSlices(p).count()));
metaClient.reloadActiveTimeline();
SyncableFileSystemView newView = getFileSystemView(metaClient);
areViewsConsistent(view, newView, 0L);
// Add 1 non-empty ingestions to COW table
Map<String, List<String>> instantsToFiles = testMultipleWriteSteps(view, Arrays.asList("12"));
// ADD replace instants
testMultipleReplaceSteps(instantsToFiles, view, Arrays.asList("13", "14"), NUM_FILE_IDS_PER_PARTITION);
// restore instants in reverse order till we rollback all replace instants
testRestore(view, Arrays.asList("15", "16"), instantsToFiles,
Arrays.asList(getHoodieReplaceInstant("14"), getHoodieReplaceInstant("13")),
"17", true, 1, fileIdsPerPartition.size());
// clear files from inmemory view for replaced instants
instantsToFiles.remove("14");
instantsToFiles.remove("13");
// add few more replace instants
testMultipleReplaceSteps(instantsToFiles, view, Arrays.asList("18", "19", "20"), NUM_FILE_IDS_PER_PARTITION);
// Clean instants.
testCleans(view, Arrays.asList("21", "22"), instantsToFiles, Arrays.asList("18", "19"), NUM_FILE_IDS_PER_PARTITION, 1);
}
private void testMultipleReplaceSteps(Map<String, List<String>> instantsToFiles, SyncableFileSystemView view, List<String> instants,
int initialExpectedSlicesPerPartition) {
int expectedSlicesPerPartition = initialExpectedSlicesPerPartition;
for (int i = 0; i < instants.size(); i++) {
try {
generateReplaceInstant(instants.get(i), instantsToFiles);
view.sync();
metaClient.reloadActiveTimeline();
SyncableFileSystemView newView = getFileSystemView(metaClient);
// 1 fileId is replaced for every partition, so subtract partitions.size()
expectedSlicesPerPartition = expectedSlicesPerPartition + fileIdsPerPartition.size() - 1;
areViewsConsistent(view, newView, expectedSlicesPerPartition * partitions.size());
} catch (IOException e) {
throw new HoodieIOException("unable to test replace", e);
}
}
}
private Map<String, List<String>> generateReplaceInstant(String replaceInstant, Map<String, List<String>> instantsToFiles) throws IOException {
Map<String, List<String>> partitionToReplacedFileIds = pickFilesToReplace(instantsToFiles);
// generate new fileIds for replace
List<String> newFileIdsToUse = IntStream.range(0, NUM_FILE_IDS_PER_PARTITION).mapToObj(x -> UUID.randomUUID().toString()).collect(Collectors.toList());
List<String> replacedFiles = addReplaceInstant(metaClient, replaceInstant,
generateDataForInstant(replaceInstant, replaceInstant, false, newFileIdsToUse),
partitionToReplacedFileIds);
instantsToFiles.put(replaceInstant, replacedFiles);
return partitionToReplacedFileIds;
}
// pick one fileId from each partition to replace and remove it from 'instantsToFiles'
private Map<String, List<String>> pickFilesToReplace(Map<String, List<String>> instantsToFiles) {
if (instantsToFiles.isEmpty()) {
return Collections.emptyMap();
}
String maxInstant = instantsToFiles.keySet().stream().max(Comparator.naturalOrder()).get();
Map<String, List<String>> partitionToFileIdsList = instantsToFiles.get(maxInstant).stream().map(file -> {
int lastPartition = file.lastIndexOf("/");
return Pair.of(file.substring(0, lastPartition), file.substring(lastPartition + 1));
}).collect(Collectors.groupingBy(Pair::getKey, Collectors.mapping(Pair::getValue, Collectors.toList())));
return partitions.stream()
.map(p -> Pair.of(p, FSUtils.getFileId(partitionToFileIdsList.get(p).get(0))))
.collect(Collectors.groupingBy(Pair::getKey, Collectors.mapping(Pair::getValue, Collectors.toList())));
}
private HoodieInstant getHoodieReplaceInstant(String timestamp) {
return new HoodieInstant(false, HoodieTimeline.REPLACE_COMMIT_ACTION, timestamp);
}
private HoodieInstant getHoodieCommitInstant(String timestamp, boolean isDeltaCommit) {
String action = isDeltaCommit ? HoodieTimeline.DELTA_COMMIT_ACTION : HoodieTimeline.COMMIT_ACTION;
return new HoodieInstant(false, action, timestamp);
}
/**
@@ -263,7 +370,7 @@ public class TestIncrementalFSViewSync extends HoodieCommonTestHarness {
put("11", Arrays.asList("12", "13", "14"));
}
},
instantsToFiles, Collections.singletonList("11"));
instantsToFiles, Collections.singletonList("11"), 0, 0);
scheduleCompaction(view2, "20");
instantsToFiles.putAll(testMultipleWriteSteps(view2, Arrays.asList("21", "22"), true, "20", 2));
// Compaction
@@ -280,8 +387,8 @@ public class TestIncrementalFSViewSync extends HoodieCommonTestHarness {
/*
* Case where multiple restores and ingestions happened
*/
testRestore(view2, Collections.singletonList("25"), true, new HashMap<>(), Collections.singletonList("24"), "29", true);
testRestore(view2, Collections.singletonList("26"), true, new HashMap<>(), Collections.singletonList("23"), "29", false);
testRestore(view2, Collections.singletonList("25"), new HashMap<>(), Collections.singletonList(getHoodieCommitInstant("24", true)), "29", true);
testRestore(view2, Collections.singletonList("26"), new HashMap<>(), Collections.singletonList(getHoodieCommitInstant("23", true)), "29", false);
instantsToFiles.putAll(testMultipleWriteSteps(view2, Collections.singletonList("27"), true, "20", 2));
scheduleCompaction(view2, "28");
instantsToFiles.putAll(testMultipleWriteSteps(view2, Collections.singletonList("29"), true, "28", 3));
@@ -304,10 +411,11 @@ public class TestIncrementalFSViewSync extends HoodieCommonTestHarness {
* Helper to run one or more rounds of cleaning, incrementally syncing the view and then validate.
*/
private void testCleans(SyncableFileSystemView view, List<String> newCleanerInstants,
Map<String, List<String>> instantsToFiles, List<String> cleanedInstants) {
Map<String, List<String>> instantsToFiles, List<String> cleanedInstants, int numberOfFilesAddedPerInstant,
int numberOfFilesReplacedPerInstant) {
Map<String, List<String>> deltaInstantMap = cleanedInstants.stream().map(e -> Pair.of(e, new ArrayList()))
.collect(Collectors.toMap(Pair::getKey, Pair::getValue));
testCleans(view, newCleanerInstants, deltaInstantMap, instantsToFiles, cleanedInstants);
testCleans(view, newCleanerInstants, deltaInstantMap, instantsToFiles, cleanedInstants, numberOfFilesAddedPerInstant, numberOfFilesReplacedPerInstant);
}
/**
@@ -321,7 +429,8 @@ public class TestIncrementalFSViewSync extends HoodieCommonTestHarness {
*/
private void testCleans(SyncableFileSystemView view, List<String> newCleanerInstants,
Map<String, List<String>> deltaInstantMap, Map<String, List<String>> instantsToFiles,
List<String> cleanedInstants) {
List<String> cleanedInstants, int numFilesAddedPerInstant, int numFilesReplacedPerInstant) {
final int netFilesAddedPerInstant = numFilesAddedPerInstant - numFilesReplacedPerInstant;
assertEquals(newCleanerInstants.size(), cleanedInstants.size());
long exp = partitions.stream().mapToLong(p1 -> view.getAllFileSlices(p1).count()).findAny().getAsLong();
LOG.info("Initial File Slices :" + exp);
@@ -333,7 +442,7 @@ public class TestIncrementalFSViewSync extends HoodieCommonTestHarness {
performClean(instant, filesToDelete, newCleanerInstants.get(idx));
exp -= fileIdsPerPartition.size();
exp -= fileIdsPerPartition.size() - numFilesReplacedPerInstant;
final long expTotalFileSlicesPerPartition = exp;
view.sync();
assertTrue(view.getLastInstant().isPresent());
@@ -345,7 +454,8 @@ public class TestIncrementalFSViewSync extends HoodieCommonTestHarness {
LOG.info("\tFileSlices :" + view.getAllFileSlices(p).collect(Collectors.toList()));
});
partitions.forEach(p -> assertEquals(fileIdsPerPartition.size(), view.getLatestFileSlices(p).count()));
final int instantIdx = newCleanerInstants.size() - idx;
partitions.forEach(p -> assertEquals(fileIdsPerPartition.size() + instantIdx * netFilesAddedPerInstant, view.getLatestFileSlices(p).count()));
partitions.forEach(p -> assertEquals(expTotalFileSlicesPerPartition, view.getAllFileSlices(p).count()));
metaClient.reloadActiveTimeline();
@@ -362,22 +472,30 @@ public class TestIncrementalFSViewSync extends HoodieCommonTestHarness {
*
* @param view Hoodie View
* @param newRestoreInstants Restore Instants
* @param isDeltaCommit is Delta Commit ?
* @param instantsToFiles List of files associated with each instant
* @param rolledBackInstants List of rolled-back instants
* @param emptyRestoreInstant Restore instant at which table becomes empty
*/
private void testRestore(SyncableFileSystemView view, List<String> newRestoreInstants, boolean isDeltaCommit,
Map<String, List<String>> instantsToFiles, List<String> rolledBackInstants, String emptyRestoreInstant,
boolean isRestore) {
private void testRestore(SyncableFileSystemView view, List<String> newRestoreInstants,
Map<String, List<String>> instantsToFiles, List<HoodieInstant> rolledBackInstants, String emptyRestoreInstant,
boolean isRestore) {
testRestore(view, newRestoreInstants, instantsToFiles, rolledBackInstants, emptyRestoreInstant, isRestore, 0, 0);
}
private void testRestore(SyncableFileSystemView view, List<String> newRestoreInstants,
Map<String, List<String>> instantsToFiles, List<HoodieInstant> rolledBackInstants, String emptyRestoreInstant,
boolean isRestore, int totalReplacedFileSlicesPerPartition, int totalFilesAddedPerPartitionPerInstant) {
assertEquals(newRestoreInstants.size(), rolledBackInstants.size());
long initialFileSlices = partitions.stream().mapToLong(p -> view.getAllFileSlices(p).count()).findAny().getAsLong();
final int numFileSlicesAddedPerInstant = (totalFilesAddedPerPartitionPerInstant - totalReplacedFileSlicesPerPartition);
final long expectedLatestFileSlices = fileIdsPerPartition.size() + (rolledBackInstants.size()) * numFileSlicesAddedPerInstant;
IntStream.range(0, newRestoreInstants.size()).forEach(idx -> {
String instant = rolledBackInstants.get(idx);
HoodieInstant instant = rolledBackInstants.get(idx);
try {
performRestore(instant, instantsToFiles.get(instant), newRestoreInstants.get(idx), isRestore);
boolean isDeltaCommit = HoodieTimeline.DELTA_COMMIT_ACTION.equalsIgnoreCase(instant.getAction());
performRestore(instant, instantsToFiles.get(instant.getTimestamp()), newRestoreInstants.get(idx), isRestore);
final long expTotalFileSlicesPerPartition =
isDeltaCommit ? initialFileSlices : initialFileSlices - ((idx + 1) * fileIdsPerPartition.size());
isDeltaCommit ? initialFileSlices : initialFileSlices - ((idx + 1) * (fileIdsPerPartition.size() - totalReplacedFileSlicesPerPartition));
view.sync();
assertTrue(view.getLastInstant().isPresent());
LOG.info("Last Instant is :" + view.getLastInstant().get());
@@ -391,7 +509,7 @@ public class TestIncrementalFSViewSync extends HoodieCommonTestHarness {
)) {
partitions.forEach(p -> assertEquals(0, view.getLatestFileSlices(p).count()));
} else {
partitions.forEach(p -> assertEquals(fileIdsPerPartition.size(), view.getLatestFileSlices(p).count()));
partitions.forEach(p -> assertEquals(expectedLatestFileSlices - (idx + 1) * numFileSlicesAddedPerInstant, view.getLatestFileSlices(p).count()));
}
partitions.forEach(p -> assertEquals(expTotalFileSlicesPerPartition, view.getAllFileSlices(p).count()));
@@ -432,28 +550,23 @@ public class TestIncrementalFSViewSync extends HoodieCommonTestHarness {
* @param files List of files to be deleted as part of rollback
* @param rollbackInstant Restore Instant
*/
private void performRestore(String instant, List<String> files, String rollbackInstant,
private void performRestore(HoodieInstant instant, List<String> files, String rollbackInstant,
boolean isRestore) throws IOException {
Map<String, List<String>> partititonToFiles = deleteFiles(files);
List<HoodieRollbackStat> rollbackStats = partititonToFiles.entrySet().stream().map(e ->
new HoodieRollbackStat(e.getKey(), e.getValue(), new ArrayList<>(), new HashMap<>())
).collect(Collectors.toList());
List<String> rollbacks = new ArrayList<>();
List<HoodieInstant> rollbacks = new ArrayList<>();
rollbacks.add(instant);
HoodieRollbackMetadata rollbackMetadata =
TimelineMetadataUtils.convertRollbackMetadata(rollbackInstant, Option.empty(), rollbacks, rollbackStats);
if (isRestore) {
HoodieRestoreMetadata metadata = new HoodieRestoreMetadata();
List<HoodieRollbackMetadata> rollbackM = new ArrayList<>();
rollbackM.add(rollbackMetadata);
metadata.setHoodieRestoreMetadata(CollectionUtils.createImmutableMap(rollbackInstant, rollbackM));
List<String> rollbackInstants = new ArrayList<>();
rollbackInstants.add(rollbackInstant);
metadata.setInstantsToRollback(rollbackInstants);
metadata.setStartRestoreTime(rollbackInstant);
HoodieRestoreMetadata metadata = TimelineMetadataUtils.convertRestoreMetadata(rollbackInstant,
100, Collections.singletonList(instant), CollectionUtils.createImmutableMap(rollbackInstant, rollbackM));
HoodieInstant restoreInstant = new HoodieInstant(true, HoodieTimeline.RESTORE_ACTION, rollbackInstant);
metaClient.getActiveTimeline().createNewInstant(restoreInstant);
@@ -465,6 +578,8 @@ public class TestIncrementalFSViewSync extends HoodieCommonTestHarness {
new HoodieInstant(true, HoodieTimeline.ROLLBACK_ACTION, rollbackInstant),
TimelineMetadataUtils.serializeRollbackMetadata(rollbackMetadata));
}
boolean deleted = metaClient.getFs().delete(new Path(metaClient.getMetaPath(), instant.getFileName()), false);
assertTrue(deleted);
}
/**
@@ -651,10 +766,7 @@ public class TestIncrementalFSViewSync extends HoodieCommonTestHarness {
private void areViewsConsistent(SyncableFileSystemView view1, SyncableFileSystemView view2,
long expectedTotalFileSlices) {
// Timeline check
HoodieTimeline timeline1 = view1.getTimeline();
HoodieTimeline timeline2 = view2.getTimeline();
assertEquals(view1.getLastInstant(), view2.getLastInstant());
CollectionUtils.elementsEqual(timeline1.getInstants().iterator(), timeline2.getInstants().iterator());
// View Checks
Map<HoodieFileGroupId, HoodieFileGroup> fileGroupsMap1 = partitions.stream().flatMap(view1::getAllFileGroups)
@@ -702,14 +814,17 @@ public class TestIncrementalFSViewSync extends HoodieCommonTestHarness {
assertEquals(ops1, ops2);
}
private List<String> addInstant(HoodieTableMetaClient metaClient, String instant, boolean deltaCommit,
String baseInstant) throws IOException {
List<Pair<String, HoodieWriteStat>> writeStats = partitions.stream().flatMap(p -> fileIdsPerPartition.stream().map(f -> {
private List<Pair<String, HoodieWriteStat>> generateDataForInstant(String baseInstant, String instant, boolean deltaCommit) {
return generateDataForInstant(baseInstant, instant, deltaCommit, fileIdsPerPartition);
}
private List<Pair<String, HoodieWriteStat>> generateDataForInstant(String baseInstant, String instant, boolean deltaCommit, List<String> fileIds) {
return partitions.stream().flatMap(p -> fileIds.stream().map(f -> {
try {
File file = new File(basePath + "/" + p + "/"
+ (deltaCommit
? FSUtils.makeLogFileName(f, ".log", baseInstant, Integer.parseInt(instant), TEST_WRITE_TOKEN)
: FSUtils.makeDataFileName(instant, TEST_WRITE_TOKEN, f)));
? FSUtils.makeLogFileName(f, ".log", baseInstant, Integer.parseInt(instant), TEST_WRITE_TOKEN)
: FSUtils.makeDataFileName(instant, TEST_WRITE_TOKEN, f)));
file.createNewFile();
HoodieWriteStat w = new HoodieWriteStat();
w.setFileId(f);
@@ -719,7 +834,11 @@ public class TestIncrementalFSViewSync extends HoodieCommonTestHarness {
throw new HoodieException(e);
}
})).collect(Collectors.toList());
}
private List<String> addInstant(HoodieTableMetaClient metaClient, String instant, boolean deltaCommit,
String baseInstant) throws IOException {
List<Pair<String, HoodieWriteStat>> writeStats = generateDataForInstant(baseInstant, instant, deltaCommit);
HoodieCommitMetadata metadata = new HoodieCommitMetadata();
writeStats.forEach(e -> metadata.addWriteStat(e.getKey(), e.getValue()));
HoodieInstant inflightInstant = new HoodieInstant(true,
@@ -735,6 +854,19 @@ public class TestIncrementalFSViewSync extends HoodieCommonTestHarness {
return writeStats.stream().map(e -> e.getValue().getPath()).collect(Collectors.toList());
}
private List<String> addReplaceInstant(HoodieTableMetaClient metaClient, String instant,
List<Pair<String, HoodieWriteStat>> writeStats,
Map<String, List<String>> partitionToReplaceFileIds) throws IOException {
HoodieReplaceCommitMetadata replaceCommitMetadata = new HoodieReplaceCommitMetadata();
writeStats.forEach(e -> replaceCommitMetadata.addWriteStat(e.getKey(), e.getValue()));
replaceCommitMetadata.setPartitionToReplaceFileIds(partitionToReplaceFileIds);
HoodieInstant inflightInstant = new HoodieInstant(true, HoodieTimeline.REPLACE_COMMIT_ACTION, instant);
metaClient.getActiveTimeline().createNewInstant(inflightInstant);
metaClient.getActiveTimeline().saveAsComplete(inflightInstant,
Option.of(replaceCommitMetadata.toJsonString().getBytes(StandardCharsets.UTF_8)));
return writeStats.stream().map(e -> e.getValue().getPath()).collect(Collectors.toList());
}
@Override
protected HoodieTableType getTableType() {
return HoodieTableType.MERGE_ON_READ;

View File

@@ -19,9 +19,11 @@
package org.apache.hudi.common.testutils;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.HoodieFileFormat;
import org.apache.hudi.common.model.HoodiePartitionMetadata;
import org.apache.hudi.common.model.HoodieReplaceCommitMetadata;
import org.apache.hudi.common.model.IOType;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
@@ -29,10 +31,9 @@ import org.apache.hudi.common.table.view.HoodieTableFileSystemView;
import org.apache.hudi.common.table.view.TableFileSystemView;
import org.apache.hudi.exception.HoodieException;
import org.apache.hadoop.fs.FileSystem;
import java.io.IOException;
import java.io.RandomAccessFile;
import java.nio.charset.StandardCharsets;
import java.nio.file.Files;
import java.nio.file.Path;
import java.nio.file.Paths;
@@ -76,6 +77,15 @@ public class FileCreateUtils {
}
}
private static void createMetaFile(String basePath, String instantTime, String suffix, byte[] content) throws IOException {
Path parentPath = Paths.get(basePath, HoodieTableMetaClient.METAFOLDER_NAME);
Files.createDirectories(parentPath);
Path metaFilePath = parentPath.resolve(instantTime + suffix);
if (Files.notExists(metaFilePath)) {
Files.write(metaFilePath, content);
}
}
public static void createCommit(String basePath, String instantTime) throws IOException {
createMetaFile(basePath, instantTime, HoodieTimeline.COMMIT_EXTENSION);
}
@@ -100,6 +110,18 @@ public class FileCreateUtils {
createMetaFile(basePath, instantTime, HoodieTimeline.INFLIGHT_DELTA_COMMIT_EXTENSION);
}
public static void createReplaceCommit(String basePath, String instantTime, HoodieReplaceCommitMetadata metadata) throws IOException {
createMetaFile(basePath, instantTime, HoodieTimeline.REPLACE_COMMIT_EXTENSION, metadata.toJsonString().getBytes(StandardCharsets.UTF_8));
}
public static void createRequestedReplaceCommit(String basePath, String instantTime) throws IOException {
createMetaFile(basePath, instantTime, HoodieTimeline.REQUESTED_REPLACE_COMMIT_EXTENSION);
}
public static void createInflightReplaceCommit(String basePath, String instantTime) throws IOException {
createMetaFile(basePath, instantTime, HoodieTimeline.INFLIGHT_REPLACE_COMMIT_EXTENSION);
}
private static void createAuxiliaryMetaFile(String basePath, String instantTime, String suffix) throws IOException {
Path parentPath = Paths.get(basePath, HoodieTableMetaClient.AUXILIARYFOLDER_NAME);
Files.createDirectories(parentPath);

View File

@@ -330,30 +330,42 @@ public class HoodieTestDataGenerator {
}
public static void createCommitFile(String basePath, String instantTime, Configuration configuration) {
HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata();
createCommitFile(basePath, instantTime, configuration, commitMetadata);
}
public static void createCommitFile(String basePath, String instantTime, Configuration configuration, HoodieCommitMetadata commitMetadata) {
Arrays.asList(HoodieTimeline.makeCommitFileName(instantTime), HoodieTimeline.makeInflightCommitFileName(instantTime),
HoodieTimeline.makeRequestedCommitFileName(instantTime))
.forEach(f -> {
Path commitFile = new Path(
basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + f);
FSDataOutputStream os = null;
try {
FileSystem fs = FSUtils.getFs(basePath, configuration);
os = fs.create(commitFile, true);
HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata();
// Write empty commit metadata
os.writeBytes(new String(commitMetadata.toJsonString().getBytes(StandardCharsets.UTF_8)));
} catch (IOException ioe) {
throw new HoodieIOException(ioe.getMessage(), ioe);
} finally {
if (null != os) {
try {
os.close();
} catch (IOException e) {
throw new HoodieIOException(e.getMessage(), e);
}
}
}
});
.forEach(f -> createMetadataFile(f, basePath, configuration, commitMetadata));
}
private static void createMetadataFile(String f, String basePath, Configuration configuration, HoodieCommitMetadata commitMetadata) {
Path commitFile = new Path(
basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + f);
FSDataOutputStream os = null;
try {
FileSystem fs = FSUtils.getFs(basePath, configuration);
os = fs.create(commitFile, true);
// Write empty commit metadata
os.writeBytes(new String(commitMetadata.toJsonString().getBytes(StandardCharsets.UTF_8)));
} catch (IOException ioe) {
throw new HoodieIOException(ioe.getMessage(), ioe);
} finally {
if (null != os) {
try {
os.close();
} catch (IOException e) {
throw new HoodieIOException(e.getMessage(), e);
}
}
}
}
public static void createReplaceFile(String basePath, String instantTime, Configuration configuration, HoodieCommitMetadata commitMetadata) {
Arrays.asList(HoodieTimeline.makeReplaceFileName(instantTime), HoodieTimeline.makeInflightReplaceFileName(instantTime),
HoodieTimeline.makeRequestedReplaceFileName(instantTime))
.forEach(f -> createMetadataFile(f, basePath, configuration, commitMetadata));
}
public static void createEmptyCleanRequestedFile(String basePath, String instantTime, Configuration configuration)
@@ -382,9 +394,9 @@ public class HoodieTestDataGenerator {
new Path(basePath + "/" + HoodieTableMetaClient.AUXILIARYFOLDER_NAME + "/" + instant.getFileName());
FileSystem fs = FSUtils.getFs(basePath, configuration);
try (FSDataOutputStream os = fs.create(commitFile, true)) {
HoodieCompactionPlan workload = new HoodieCompactionPlan();
HoodieCompactionPlan workload = HoodieCompactionPlan.newBuilder().setVersion(1).build();
// Write empty commit metadata
os.writeBytes(new String(TimelineMetadataUtils.serializeCompactionPlan(workload).get(), StandardCharsets.UTF_8));
os.write(TimelineMetadataUtils.serializeCompactionPlan(workload).get());
}
}

View File

@@ -19,6 +19,7 @@
package org.apache.hudi.common.testutils;
import org.apache.hudi.common.model.HoodieReplaceCommitMetadata;
import org.apache.hudi.common.model.IOType;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
@@ -49,10 +50,13 @@ import static org.apache.hudi.common.testutils.FileCreateUtils.createDeltaCommit
import static org.apache.hudi.common.testutils.FileCreateUtils.createInflightCommit;
import static org.apache.hudi.common.testutils.FileCreateUtils.createInflightCompaction;
import static org.apache.hudi.common.testutils.FileCreateUtils.createInflightDeltaCommit;
import static org.apache.hudi.common.testutils.FileCreateUtils.createInflightReplaceCommit;
import static org.apache.hudi.common.testutils.FileCreateUtils.createMarkerFile;
import static org.apache.hudi.common.testutils.FileCreateUtils.createReplaceCommit;
import static org.apache.hudi.common.testutils.FileCreateUtils.createRequestedCommit;
import static org.apache.hudi.common.testutils.FileCreateUtils.createRequestedCompaction;
import static org.apache.hudi.common.testutils.FileCreateUtils.createRequestedDeltaCommit;
import static org.apache.hudi.common.testutils.FileCreateUtils.createRequestedReplaceCommit;
import static org.apache.hudi.common.testutils.FileCreateUtils.logFileName;
public class HoodieTestTable {
@@ -145,6 +149,15 @@ public class HoodieTestTable {
return this;
}
public HoodieTestTable addReplaceCommit(String instantTime, HoodieReplaceCommitMetadata metadata) throws Exception {
createRequestedReplaceCommit(basePath, instantTime);
createInflightReplaceCommit(basePath, instantTime);
createReplaceCommit(basePath, instantTime, metadata);
currentInstantTime = instantTime;
metaClient = HoodieTableMetaClient.reload(metaClient);
return this;
}
public HoodieTestTable addRequestedCompaction(String instantTime) throws IOException {
createRequestedCompaction(basePath, instantTime);
currentInstantTime = instantTime;

View File

@@ -18,6 +18,21 @@
package org.apache.hudi.common.testutils;
import com.esotericsoftware.kryo.Kryo;
import com.esotericsoftware.kryo.io.Input;
import com.esotericsoftware.kryo.io.Output;
import com.esotericsoftware.kryo.serializers.JavaSerializer;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.LocatedFileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.RemoteIterator;
import org.apache.hadoop.util.StringUtils;
import org.apache.hudi.avro.HoodieAvroUtils;
import org.apache.hudi.avro.model.HoodieActionInstant;
import org.apache.hudi.avro.model.HoodieCleanMetadata;
@@ -52,22 +67,6 @@ import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.exception.HoodieIOException;
import com.esotericsoftware.kryo.Kryo;
import com.esotericsoftware.kryo.io.Input;
import com.esotericsoftware.kryo.io.Output;
import com.esotericsoftware.kryo.serializers.JavaSerializer;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.LocatedFileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.RemoteIterator;
import org.apache.hadoop.util.StringUtils;
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.File;

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.HoodieWriteStat;
import org.apache.hudi.common.model.WriteOperationType;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.junit.jupiter.api.Test;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.TRIP_SCHEMA;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertTrue;
public class TestCommitUtils {
@Test
public void testCommitMetadataCreation() {
List<HoodieWriteStat> writeStats = new ArrayList<>();
writeStats.add(createWriteStat("p1", "f1"));
writeStats.add(createWriteStat("p2", "f2"));
Map<String, List<String>> partitionToReplaceFileIds = new HashMap<>();
List<String> replacedFileIds = new ArrayList<>();
replacedFileIds.add("f0");
partitionToReplaceFileIds.put("p1", replacedFileIds);
HoodieCommitMetadata commitMetadata = CommitUtils.buildMetadata(writeStats, partitionToReplaceFileIds,
Option.empty(),
WriteOperationType.INSERT,
TRIP_SCHEMA,
HoodieTimeline.DELTA_COMMIT_ACTION);
assertFalse(commitMetadata instanceof HoodieReplaceCommitMetadata);
assertEquals(2, commitMetadata.getPartitionToWriteStats().size());
assertEquals("f1", commitMetadata.getPartitionToWriteStats().get("p1").get(0).getFileId());
assertEquals("f2", commitMetadata.getPartitionToWriteStats().get("p2").get(0).getFileId());
assertEquals(WriteOperationType.INSERT, commitMetadata.getOperationType());
assertEquals(TRIP_SCHEMA, commitMetadata.getMetadata(HoodieCommitMetadata.SCHEMA_KEY));
}
@Test
public void testReplaceMetadataCreation() {
List<HoodieWriteStat> writeStats = new ArrayList<>();
writeStats.add(createWriteStat("p1", "f1"));
writeStats.add(createWriteStat("p2", "f2"));
Map<String, List<String>> partitionToReplaceFileIds = new HashMap<>();
List<String> replacedFileIds = new ArrayList<>();
replacedFileIds.add("f0");
partitionToReplaceFileIds.put("p1", replacedFileIds);
HoodieCommitMetadata commitMetadata = CommitUtils.buildMetadata(writeStats, partitionToReplaceFileIds,
Option.empty(),
WriteOperationType.INSERT,
TRIP_SCHEMA,
HoodieTimeline.REPLACE_COMMIT_ACTION);
assertTrue(commitMetadata instanceof HoodieReplaceCommitMetadata);
HoodieReplaceCommitMetadata replaceCommitMetadata = (HoodieReplaceCommitMetadata) commitMetadata;
assertEquals(1, replaceCommitMetadata.getPartitionToReplaceFileIds().size());
assertEquals("f0", replaceCommitMetadata.getPartitionToReplaceFileIds().get("p1").get(0));
assertEquals(2, commitMetadata.getPartitionToWriteStats().size());
assertEquals("f1", commitMetadata.getPartitionToWriteStats().get("p1").get(0).getFileId());
assertEquals("f2", commitMetadata.getPartitionToWriteStats().get("p2").get(0).getFileId());
assertEquals(WriteOperationType.INSERT, commitMetadata.getOperationType());
assertEquals(TRIP_SCHEMA, commitMetadata.getMetadata(HoodieCommitMetadata.SCHEMA_KEY));
}
private HoodieWriteStat createWriteStat(String partition, String fileId) {
HoodieWriteStat writeStat1 = new HoodieWriteStat();
writeStat1.setPartitionPath(partition);
writeStat1.setFileId(fileId);
return writeStat1;
}
}