1
0

[HUDI-308] Avoid Renames for tracking state transitions of all actions on dataset

This commit is contained in:
Balaji Varadarajan
2019-12-04 01:02:17 -08:00
committed by Balaji Varadarajan
parent 8963a68e6a
commit 9a1f698eef
47 changed files with 1121 additions and 403 deletions

View File

@@ -37,6 +37,7 @@
"default": null
},
{
/** DEPRECATED **/
"name":"hoodieCompactionMetadata",
"type":[
"null",
@@ -74,6 +75,27 @@
"name":"version",
"type":["int", "null"],
"default": 1
},
{
"name":"hoodieCompactionPlan",
"type":[
"null",
"HoodieCompactionPlan"
],
"default": null
},
{
"name":"hoodieCleanerPlan",
"type":[
"null",
"HoodieCleanerPlan"
],
"default": null
},
{
"name":"actionState",
"type":["null","string"],
"default": null
}
]
}

View File

@@ -0,0 +1,79 @@
/*
* 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.google.common.base.Preconditions;
import java.io.Serializable;
import java.util.Objects;
/**
* Metadata Layout Version. Add new version when timeline format changes
*/
public class TimelineLayoutVersion implements Serializable, Comparable<TimelineLayoutVersion> {
public static final Integer VERSION_0 = 0; // pre 0.5.1 version format
public static final Integer VERSION_1 = 1; // current version with no renames
public static final Integer CURR_VERSION = VERSION_1;
public static final TimelineLayoutVersion CURR_LAYOUT_VERSION = new TimelineLayoutVersion(CURR_VERSION);
private Integer version;
public TimelineLayoutVersion(Integer version) {
Preconditions.checkArgument(version <= CURR_VERSION);
Preconditions.checkArgument(version >= VERSION_0);
this.version = version;
}
/**
* For Pre 0.5.1 release, there was no metadata version. This method is used to detect
* this case.
* @return
*/
public boolean isNullVersion() {
return Objects.equals(version, VERSION_0);
}
public Integer getVersion() {
return version;
}
@Override
public boolean equals(Object o) {
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
TimelineLayoutVersion that = (TimelineLayoutVersion) o;
return Objects.equals(version, that.version);
}
@Override
public int hashCode() {
return Objects.hash(version);
}
@Override
public int compareTo(TimelineLayoutVersion o) {
return Integer.compare(version, o.version);
}
}

View File

@@ -21,6 +21,7 @@ package org.apache.hudi.common.table;
import org.apache.hudi.common.model.HoodieAvroPayload;
import org.apache.hudi.common.model.HoodieFileFormat;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.model.TimelineLayoutVersion;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.hadoop.fs.FSDataInputStream;
@@ -54,12 +55,15 @@ public class HoodieTableConfig implements Serializable {
public static final String HOODIE_TABLE_TYPE_PROP_NAME = "hoodie.table.type";
public static final String HOODIE_RO_FILE_FORMAT_PROP_NAME = "hoodie.table.ro.file.format";
public static final String HOODIE_RT_FILE_FORMAT_PROP_NAME = "hoodie.table.rt.file.format";
public static final String HOODIE_TIMELINE_LAYOUT_VERSION = "hoodie.timeline.layout.version";
public static final String HOODIE_PAYLOAD_CLASS_PROP_NAME = "hoodie.compaction.payload.class";
public static final String HOODIE_ARCHIVELOG_FOLDER_PROP_NAME = "hoodie.archivelog.folder";
public static final HoodieTableType DEFAULT_TABLE_TYPE = HoodieTableType.COPY_ON_WRITE;
public static final HoodieFileFormat DEFAULT_RO_FILE_FORMAT = HoodieFileFormat.PARQUET;
public static final HoodieFileFormat DEFAULT_RT_FILE_FORMAT = HoodieFileFormat.HOODIE_LOG;
public static final Integer DEFAULT_TIMELINE_LAYOUT_VERSION = TimelineLayoutVersion.VERSION_0;
public static final String DEFAULT_PAYLOAD_CLASS = HoodieAvroPayload.class.getName();
public static final String DEFAULT_ARCHIVELOG_FOLDER = "";
private Properties props;
@@ -112,6 +116,10 @@ public class HoodieTableConfig implements Serializable {
if (!properties.containsKey(HOODIE_ARCHIVELOG_FOLDER_PROP_NAME)) {
properties.setProperty(HOODIE_ARCHIVELOG_FOLDER_PROP_NAME, DEFAULT_ARCHIVELOG_FOLDER);
}
if (!properties.containsKey(HOODIE_TIMELINE_LAYOUT_VERSION)) {
// Use latest Version as default unless forced by client
properties.setProperty(HOODIE_TIMELINE_LAYOUT_VERSION, TimelineLayoutVersion.CURR_VERSION.toString());
}
properties.store(outputStream, "Properties saved on " + new Date(System.currentTimeMillis()));
}
}
@@ -126,6 +134,12 @@ public class HoodieTableConfig implements Serializable {
return DEFAULT_TABLE_TYPE;
}
public TimelineLayoutVersion getTimelineLayoutVersion() {
return new TimelineLayoutVersion(Integer.valueOf(props.getProperty(HOODIE_TIMELINE_LAYOUT_VERSION,
String.valueOf(DEFAULT_TIMELINE_LAYOUT_VERSION))));
}
/**
* Read the payload class for HoodieRecords from the table properties.
*/

View File

@@ -21,6 +21,7 @@ package org.apache.hudi.common.table;
import org.apache.hudi.common.SerializableConfiguration;
import org.apache.hudi.common.io.storage.HoodieWrapperFileSystem;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.model.TimelineLayoutVersion;
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
import org.apache.hudi.common.table.timeline.HoodieArchivedTimeline;
import org.apache.hudi.common.table.timeline.HoodieInstant;
@@ -28,6 +29,7 @@ import org.apache.hudi.common.util.ConsistencyGuardConfig;
import org.apache.hudi.common.util.FSUtils;
import org.apache.hudi.common.util.FailSafeConsistencyGuard;
import org.apache.hudi.common.util.NoOpConsistencyGuard;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.exception.DatasetNotFoundException;
import org.apache.hudi.exception.HoodieException;
@@ -44,12 +46,12 @@ import java.io.File;
import java.io.IOException;
import java.io.Serializable;
import java.util.Arrays;
import java.util.Comparator;
import java.util.List;
import java.util.Objects;
import java.util.Properties;
import java.util.Set;
import java.util.stream.Collectors;
import java.util.stream.Stream;
/**
* <code>HoodieTableMetaClient</code> allows to access meta-data about a hoodie table It returns meta-data about
@@ -76,6 +78,7 @@ public class HoodieTableMetaClient implements Serializable {
private boolean loadActiveTimelineOnLoad;
private SerializableConfiguration hadoopConf;
private HoodieTableType tableType;
private TimelineLayoutVersion timelineLayoutVersion;
private HoodieTableConfig tableConfig;
private HoodieActiveTimeline activeTimeline;
private HoodieArchivedTimeline archivedTimeline;
@@ -87,11 +90,13 @@ public class HoodieTableMetaClient implements Serializable {
}
public HoodieTableMetaClient(Configuration conf, String basePath, boolean loadActiveTimelineOnLoad) {
this(conf, basePath, loadActiveTimelineOnLoad, ConsistencyGuardConfig.newBuilder().build());
this(conf, basePath, loadActiveTimelineOnLoad, ConsistencyGuardConfig.newBuilder().build(),
Option.of(TimelineLayoutVersion.CURR_LAYOUT_VERSION));
}
public HoodieTableMetaClient(Configuration conf, String basePath, boolean loadActiveTimelineOnLoad,
ConsistencyGuardConfig consistencyGuardConfig) throws DatasetNotFoundException {
ConsistencyGuardConfig consistencyGuardConfig, Option<TimelineLayoutVersion> layoutVersion)
throws DatasetNotFoundException {
LOG.info("Loading HoodieTableMetaClient from " + basePath);
this.basePath = basePath;
this.consistencyGuardConfig = consistencyGuardConfig;
@@ -103,8 +108,9 @@ public class HoodieTableMetaClient implements Serializable {
DatasetNotFoundException.checkValidDataset(fs, basePathDir, metaPathDir);
this.tableConfig = new HoodieTableConfig(fs, metaPath);
this.tableType = tableConfig.getTableType();
LOG.info("Finished Loading Table of type " + tableType + " from " + basePath);
this.timelineLayoutVersion = layoutVersion.orElse(tableConfig.getTimelineLayoutVersion());
this.loadActiveTimelineOnLoad = loadActiveTimelineOnLoad;
LOG.info("Finished Loading Table of type " + tableType + "(version=" + timelineLayoutVersion + ") from " + basePath);
if (loadActiveTimelineOnLoad) {
LOG.info("Loading Active commit timeline for " + basePath);
getActiveTimeline();
@@ -120,7 +126,8 @@ public class HoodieTableMetaClient implements Serializable {
public static HoodieTableMetaClient reload(HoodieTableMetaClient oldMetaClient) {
return new HoodieTableMetaClient(oldMetaClient.hadoopConf.get(), oldMetaClient.basePath,
oldMetaClient.loadActiveTimelineOnLoad, oldMetaClient.consistencyGuardConfig);
oldMetaClient.loadActiveTimelineOnLoad, oldMetaClient.consistencyGuardConfig,
Option.of(oldMetaClient.timelineLayoutVersion));
}
/**
@@ -201,6 +208,10 @@ public class HoodieTableMetaClient implements Serializable {
return tableConfig;
}
public TimelineLayoutVersion getTimelineLayoutVersion() {
return timelineLayoutVersion;
}
/**
* Get the FS implementation for this table.
*/
@@ -274,12 +285,8 @@ public class HoodieTableMetaClient implements Serializable {
*/
public static HoodieTableMetaClient initTableType(Configuration hadoopConf, String basePath, String tableType,
String tableName, String archiveLogFolder) throws IOException {
HoodieTableType type = HoodieTableType.valueOf(tableType);
Properties properties = new Properties();
properties.put(HoodieTableConfig.HOODIE_TABLE_NAME_PROP_NAME, tableName);
properties.put(HoodieTableConfig.HOODIE_TABLE_TYPE_PROP_NAME, type.name());
properties.put(HoodieTableConfig.HOODIE_ARCHIVELOG_FOLDER_PROP_NAME, archiveLogFolder);
return HoodieTableMetaClient.initDatasetAndGetMetaClient(hadoopConf, basePath, properties);
return initTableType(hadoopConf, basePath, HoodieTableType.valueOf(tableType), tableName,
archiveLogFolder, null, null);
}
/**
@@ -287,12 +294,26 @@ public class HoodieTableMetaClient implements Serializable {
*/
public static HoodieTableMetaClient initTableType(Configuration hadoopConf, String basePath,
HoodieTableType tableType, String tableName, String payloadClassName) throws IOException {
return initTableType(hadoopConf, basePath, tableType, tableName, null, payloadClassName, null);
}
public static HoodieTableMetaClient initTableType(Configuration hadoopConf, String basePath,
HoodieTableType tableType, String tableName, String archiveLogFolder, String payloadClassName,
Integer timelineLayoutVersion) throws IOException {
Properties properties = new Properties();
properties.setProperty(HoodieTableConfig.HOODIE_TABLE_NAME_PROP_NAME, tableName);
properties.setProperty(HoodieTableConfig.HOODIE_TABLE_TYPE_PROP_NAME, tableType.name());
if (tableType == HoodieTableType.MERGE_ON_READ) {
if (tableType == HoodieTableType.MERGE_ON_READ && payloadClassName != null) {
properties.setProperty(HoodieTableConfig.HOODIE_PAYLOAD_CLASS_PROP_NAME, payloadClassName);
}
if (null != archiveLogFolder) {
properties.put(HoodieTableConfig.HOODIE_ARCHIVELOG_FOLDER_PROP_NAME, archiveLogFolder);
}
if (null != timelineLayoutVersion) {
properties.put(HoodieTableConfig.HOODIE_TIMELINE_LAYOUT_VERSION, String.valueOf(timelineLayoutVersion));
}
return HoodieTableMetaClient.initDatasetAndGetMetaClient(hadoopConf, basePath, properties);
}
@@ -414,23 +435,41 @@ public class HoodieTableMetaClient implements Serializable {
/**
* Helper method to scan all hoodie-instant metafiles and construct HoodieInstant objects.
*
* @param fs FileSystem
* @param metaPath Meta Path where hoodie instants are present
* @param includedExtensions Included hoodie extensions
* @param applyLayoutVersionFilters Depending on Timeline layout version, if there are multiple states for the same
* action instant, only include the highest state
* @return List of Hoodie Instants generated
* @throws IOException in case of failure
*/
public static List<HoodieInstant> scanHoodieInstantsFromFileSystem(FileSystem fs, Path metaPath,
Set<String> includedExtensions) throws IOException {
return Arrays.stream(HoodieTableMetaClient.scanFiles(fs, metaPath, path -> {
// Include only the meta files with extensions that needs to be included
String extension = FSUtils.getFileExtension(path.getName());
return includedExtensions.contains(extension);
})).sorted(Comparator.comparing(
// Sort the meta-data by the instant time (first part of the file name)
fileStatus -> FSUtils.getInstantTime(fileStatus.getPath().getName())))
// create HoodieInstantMarkers from FileStatus, which extracts properties
.map(HoodieInstant::new).collect(Collectors.toList());
public List<HoodieInstant> scanHoodieInstantsFromFileSystem(Set<String> includedExtensions,
boolean applyLayoutVersionFilters) throws IOException {
return scanHoodieInstantsFromFileSystem(new Path(metaPath), includedExtensions, applyLayoutVersionFilters);
}
/**
* Helper method to scan all hoodie-instant metafiles and construct HoodieInstant objects.
*
* @param timelinePath MetaPath where instant files are stored
* @param includedExtensions Included hoodie extensions
* @param applyLayoutVersionFilters Depending on Timeline layout version, if there are multiple states for the same
* action instant, only include the highest state
* @return List of Hoodie Instants generated
* @throws IOException in case of failure
*/
public List<HoodieInstant> scanHoodieInstantsFromFileSystem(Path timelinePath, Set<String> includedExtensions,
boolean applyLayoutVersionFilters) throws IOException {
Stream<HoodieInstant> instantStream = Arrays.stream(
HoodieTableMetaClient
.scanFiles(getFs(), timelinePath, path -> {
// Include only the meta files with extensions that needs to be included
String extension = FSUtils.getFileExtension(path.getName());
return includedExtensions.contains(extension);
})).map(HoodieInstant::new);
if (applyLayoutVersionFilters) {
instantStream = TimelineLayout.getLayout(getTimelineLayoutVersion()).filterHoodieInstants(instantStream);
}
return instantStream.sorted().collect(Collectors.toList());
}
@Override

View File

@@ -18,6 +18,7 @@
package org.apache.hudi.common.table;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.table.timeline.HoodieDefaultTimeline;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.timeline.HoodieInstant.State;
@@ -62,6 +63,8 @@ public interface HoodieTimeline extends Serializable {
String SAVEPOINT_EXTENSION = "." + SAVEPOINT_ACTION;
// this is to preserve backwards compatibility on commit in-flight filenames
String INFLIGHT_COMMIT_EXTENSION = INFLIGHT_EXTENSION;
String REQUESTED_COMMIT_EXTENSION = "." + COMMIT_ACTION + REQUESTED_EXTENSION;
String REQUESTED_DELTA_COMMIT_EXTENSION = "." + DELTA_COMMIT_ACTION + REQUESTED_EXTENSION;
String INFLIGHT_DELTA_COMMIT_EXTENSION = "." + DELTA_COMMIT_ACTION + INFLIGHT_EXTENSION;
String INFLIGHT_CLEAN_EXTENSION = "." + CLEAN_ACTION + INFLIGHT_EXTENSION;
String REQUESTED_CLEAN_EXTENSION = "." + CLEAN_ACTION + REQUESTED_EXTENSION;
@@ -94,7 +97,7 @@ public interface HoodieTimeline extends Serializable {
*
* @return New instance of HoodieTimeline with just in-flights excluding compaction inflights
*/
HoodieTimeline filterInflightsExcludingCompaction();
HoodieTimeline filterPendingExcludingCompaction();
/**
* Filter this timeline to just include the completed instants.
@@ -251,7 +254,17 @@ public interface HoodieTimeline extends Serializable {
return new HoodieInstant(State.INFLIGHT, COMPACTION_ACTION, timestamp);
}
static HoodieInstant getInflightInstant(final HoodieInstant instant) {
/**
* Returns the inflight instant corresponding to the instant being passed. Takes care of changes in action names
* between inflight and completed instants (compaction <=> commit).
* @param instant Hoodie Instant
* @param tableType Hoodie Table Type
* @return Inflight Hoodie Instant
*/
static HoodieInstant getInflightInstant(final HoodieInstant instant, final HoodieTableType tableType) {
if ((tableType == HoodieTableType.MERGE_ON_READ) && instant.getAction().equals(COMMIT_ACTION)) {
return new HoodieInstant(true, COMPACTION_ACTION, instant.getTimestamp());
}
return new HoodieInstant(true, instant.getAction(), instant.getTimestamp());
}
@@ -263,6 +276,10 @@ public interface HoodieTimeline extends Serializable {
return StringUtils.join(commitTime, HoodieTimeline.INFLIGHT_COMMIT_EXTENSION);
}
static String makeRequestedCommitFileName(String commitTime) {
return StringUtils.join(commitTime, HoodieTimeline.REQUESTED_COMMIT_EXTENSION);
}
static String makeCleanerFileName(String instant) {
return StringUtils.join(instant, HoodieTimeline.CLEAN_EXTENSION);
}
@@ -295,6 +312,10 @@ public interface HoodieTimeline extends Serializable {
return StringUtils.join(commitTime, HoodieTimeline.INFLIGHT_DELTA_COMMIT_EXTENSION);
}
static String makeRequestedDeltaFileName(String commitTime) {
return StringUtils.join(commitTime, HoodieTimeline.REQUESTED_DELTA_COMMIT_EXTENSION);
}
static String makeInflightCompactionFileName(String commitTime) {
return StringUtils.join(commitTime, HoodieTimeline.INFLIGHT_COMPACTION_EXTENSION);
}

View File

@@ -0,0 +1,79 @@
/*
* 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.table;
import org.apache.hudi.common.model.TimelineLayoutVersion;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.util.collection.Pair;
import java.io.Serializable;
import java.util.HashMap;
import java.util.Map;
import java.util.stream.Collectors;
import java.util.stream.Stream;
/**
* Timeline Layout responsible for applying specific filters when generating timeline instants.
*/
public abstract class TimelineLayout implements Serializable {
private static final Map<TimelineLayoutVersion, TimelineLayout> LAYOUT_MAP = new HashMap<>();
static {
LAYOUT_MAP.put(new TimelineLayoutVersion(TimelineLayoutVersion.VERSION_0), new TimelineLayoutV0());
LAYOUT_MAP.put(new TimelineLayoutVersion(TimelineLayoutVersion.VERSION_1), new TimelineLayoutV1());
}
public static TimelineLayout getLayout(TimelineLayoutVersion version) {
return LAYOUT_MAP.get(version);
}
public abstract Stream<HoodieInstant> filterHoodieInstants(Stream<HoodieInstant> instantStream);
/**
* Table Layout where state transitions are managed by renaming files.
*/
private static class TimelineLayoutV0 extends TimelineLayout {
@Override
public Stream<HoodieInstant> filterHoodieInstants(Stream<HoodieInstant> instantStream) {
return instantStream;
}
}
/**
* Table Layout where state transitions are managed by creating new files.
*/
private static class TimelineLayoutV1 extends TimelineLayout {
@Override
public Stream<HoodieInstant> filterHoodieInstants(Stream<HoodieInstant> instantStream) {
return instantStream.collect(Collectors.groupingBy(instant -> Pair.of(instant.getTimestamp(),
HoodieInstant.getComparableAction(instant.getAction())))).entrySet().stream()
.map(e -> e.getValue().stream().reduce((x, y) -> {
// Pick the one with the highest state
if (x.getState().compareTo(y.getState()) >= 0) {
return x;
}
return y;
}).get());
}
}
}

View File

@@ -43,6 +43,7 @@ import java.util.HashSet;
import java.util.Set;
import java.util.concurrent.atomic.AtomicReference;
import java.util.function.Function;
import java.util.stream.Collectors;
import java.util.stream.Stream;
/**
@@ -61,21 +62,21 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
public static final SimpleDateFormat COMMIT_FORMATTER = new SimpleDateFormat("yyyyMMddHHmmss");
public static final Set<String> VALID_EXTENSIONS_IN_ACTIVE_TIMELINE =
new HashSet<>(Arrays.asList(new String[] {COMMIT_EXTENSION, INFLIGHT_COMMIT_EXTENSION, DELTA_COMMIT_EXTENSION,
INFLIGHT_DELTA_COMMIT_EXTENSION, SAVEPOINT_EXTENSION, INFLIGHT_SAVEPOINT_EXTENSION, CLEAN_EXTENSION,
INFLIGHT_CLEAN_EXTENSION, REQUESTED_CLEAN_EXTENSION, INFLIGHT_COMPACTION_EXTENSION,
REQUESTED_COMPACTION_EXTENSION, INFLIGHT_RESTORE_EXTENSION, RESTORE_EXTENSION}));
public static final Set<String> VALID_EXTENSIONS_IN_ACTIVE_TIMELINE = new HashSet<>(Arrays.asList(
new String[]{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}));
private static final Logger LOG = LogManager.getLogger(HoodieActiveTimeline.class);
protected HoodieTableMetaClient metaClient;
private static AtomicReference<String> lastInstantTime = new AtomicReference<>(String.valueOf(Integer.MIN_VALUE));
/**
* Returns next commit time in the {@link #COMMIT_FORMATTER} format.
* Ensures each commit time is atleast 1 second apart since we create COMMIT times at second granularity
* Returns next instant time in the {@link #COMMIT_FORMATTER} format.
* Ensures each instant time is atleast 1 second apart since we create instant times at second granularity
*/
public static String createNewCommitTime() {
public static String createNewInstantTime() {
lastInstantTime.updateAndGet((oldVal) -> {
String newCommitTime = null;
do {
@@ -87,12 +88,15 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
}
protected HoodieActiveTimeline(HoodieTableMetaClient metaClient, Set<String> includedExtensions) {
this(metaClient, includedExtensions, true);
}
protected HoodieActiveTimeline(HoodieTableMetaClient metaClient, Set<String> includedExtensions,
boolean applyLayoutFilters) {
// Filter all the filter in the metapath and include only the extensions passed and
// convert them into HoodieInstant
try {
this.setInstants(HoodieTableMetaClient.scanHoodieInstantsFromFileSystem(metaClient.getFs(),
new Path(metaClient.getMetaPath()), includedExtensions));
LOG.info("Loaded instants " + getInstants());
this.setInstants(metaClient.scanHoodieInstantsFromFileSystem(includedExtensions, applyLayoutFilters));
} catch (IOException e) {
throw new HoodieIOException("Failed to scan metadata", e);
}
@@ -100,12 +104,19 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
// multiple casts will make this lambda serializable -
// http://docs.oracle.com/javase/specs/jls/se8/html/jls-15.html#jls-15.16
this.details = (Function<HoodieInstant, Option<byte[]>> & Serializable) this::getInstantDetails;
LOG.info("Loaded instants " + getInstants().collect(Collectors.toList()));
}
public HoodieActiveTimeline(HoodieTableMetaClient metaClient) {
this(metaClient, new ImmutableSet.Builder<String>().addAll(VALID_EXTENSIONS_IN_ACTIVE_TIMELINE).build());
}
public HoodieActiveTimeline(HoodieTableMetaClient metaClient, boolean applyLayoutFilter) {
this(metaClient,
new ImmutableSet.Builder<String>()
.addAll(VALID_EXTENSIONS_IN_ACTIVE_TIMELINE).build(), applyLayoutFilter);
}
/**
* For serialization and de-serialization only.
*
@@ -209,10 +220,10 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
return getInstants().filter(s -> s.getAction().equals(action));
}
public void createInflight(HoodieInstant instant) {
LOG.info("Creating a new in-flight instant " + instant);
public void createNewInstant(HoodieInstant instant) {
LOG.info("Creating a new instant " + instant);
// Create the in-flight file
createFileInMetaPath(instant.getFileName(), Option.empty());
createFileInMetaPath(instant.getFileName(), Option.empty(), false);
}
public void saveAsComplete(HoodieInstant instant, Option<byte[]> data) {
@@ -223,18 +234,12 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
LOG.info("Completed " + instant);
}
public void revertToInflight(HoodieInstant instant) {
LOG.info("Reverting " + instant + " to inflight ");
revertStateTransition(instant, HoodieTimeline.getInflightInstant(instant));
LOG.info("Reverted " + instant + " to inflight");
}
public HoodieInstant revertToRequested(HoodieInstant instant) {
LOG.warn("Reverting " + instant + " to requested ");
HoodieInstant requestedInstant = HoodieTimeline.getRequestedInstant(instant);
revertStateTransition(instant, HoodieTimeline.getRequestedInstant(instant));
LOG.warn("Reverted " + instant + " to requested");
return requestedInstant;
public HoodieInstant revertToInflight(HoodieInstant instant) {
LOG.info("Reverting instant to inflight " + instant);
HoodieInstant inflight = HoodieTimeline.getInflightInstant(instant, metaClient.getTableType());
revertCompleteToInflight(instant, inflight);
LOG.info("Reverted " + instant + " to inflight " + inflight);
return inflight;
}
public void deleteInflight(HoodieInstant instant) {
@@ -242,6 +247,11 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
deleteInstantFile(instant);
}
public void deletePending(HoodieInstant instant) {
Preconditions.checkArgument(!instant.isCompleted());
deleteInstantFile(instant);
}
public void deleteCompactionRequested(HoodieInstant instant) {
Preconditions.checkArgument(instant.isRequested());
Preconditions.checkArgument(instant.getAction() == HoodieTimeline.COMPACTION_ACTION);
@@ -254,9 +264,9 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
try {
boolean result = metaClient.getFs().delete(inFlightCommitFilePath, false);
if (result) {
LOG.info("Removed in-flight " + instant);
LOG.info("Removed instant " + instant);
} else {
throw new HoodieIOException("Could not delete in-flight instant " + instant);
throw new HoodieIOException("Could not delete instant " + instant);
}
} catch (IOException e) {
throw new HoodieIOException("Could not remove inflight commit " + inFlightCommitFilePath, e);
@@ -273,8 +283,13 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
// BEGIN - COMPACTION RELATED META-DATA MANAGEMENT.
//-----------------------------------------------------------------
public Option<byte[]> getInstantAuxiliaryDetails(HoodieInstant instant) {
Path detailPath = new Path(metaClient.getMetaAuxiliaryPath(), instant.getFileName());
public Option<byte[]> readPlanAsBytes(HoodieInstant instant) {
Path detailPath = null;
if (metaClient.getTimelineLayoutVersion().isNullVersion()) {
detailPath = new Path(metaClient.getMetaAuxiliaryPath(), instant.getFileName());
} else {
detailPath = new Path(metaClient.getMetaPath(), instant.getFileName());
}
return readDataFromPath(detailPath);
}
@@ -289,8 +304,12 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
Preconditions.checkArgument(inflightInstant.isInflight());
HoodieInstant requestedInstant =
new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, inflightInstant.getTimestamp());
// Pass empty data since it is read from the corresponding .aux/.compaction instant file
transitionState(inflightInstant, requestedInstant, Option.empty());
if (metaClient.getTimelineLayoutVersion().isNullVersion()) {
// Pass empty data since it is read from the corresponding .aux/.compaction instant file
transitionState(inflightInstant, requestedInstant, Option.empty());
} else {
deleteInflight(inflightInstant);
}
return requestedInstant;
}
@@ -325,8 +344,14 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
}
private void createFileInAuxiliaryFolder(HoodieInstant instant, Option<byte[]> data) {
Path fullPath = new Path(metaClient.getMetaAuxiliaryPath(), instant.getFileName());
createFileInPath(fullPath, data);
if (metaClient.getTimelineLayoutVersion().isNullVersion()) {
/**
* For latest version, since we write immutable files directly in timeline directory, there is no need to write
* additional immutable files in .aux folder
*/
Path fullPath = new Path(metaClient.getMetaAuxiliaryPath(), instant.getFileName());
createFileInPath(fullPath, data);
}
}
//-----------------------------------------------------------------
@@ -355,59 +380,92 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
* Transition Clean State from requested to inflight.
*
* @param requestedInstant requested instant
* @param data Optional data to be stored
* @return commit instant
*/
public HoodieInstant transitionCleanRequestedToInflight(HoodieInstant requestedInstant) {
public HoodieInstant transitionCleanRequestedToInflight(HoodieInstant requestedInstant, Option<byte[]> data) {
Preconditions.checkArgument(requestedInstant.getAction().equals(HoodieTimeline.CLEAN_ACTION));
Preconditions.checkArgument(requestedInstant.isRequested());
HoodieInstant inflight = new HoodieInstant(State.INFLIGHT, CLEAN_ACTION, requestedInstant.getTimestamp());
transitionState(requestedInstant, inflight, Option.empty());
transitionState(requestedInstant, inflight, data);
return inflight;
}
private void transitionState(HoodieInstant fromInstant, HoodieInstant toInstant, Option<byte[]> data) {
Preconditions.checkArgument(fromInstant.getTimestamp().equals(toInstant.getTimestamp()));
Path commitFilePath = new Path(metaClient.getMetaPath(), toInstant.getFileName());
try {
// Re-create the .inflight file by opening a new file and write the commit metadata in
Path inflightCommitFile = new Path(metaClient.getMetaPath(), fromInstant.getFileName());
createFileInMetaPath(fromInstant.getFileName(), data);
boolean success = metaClient.getFs().rename(inflightCommitFile, commitFilePath);
if (!success) {
throw new HoodieIOException("Could not rename " + inflightCommitFile + " to " + commitFilePath);
if (metaClient.getTimelineLayoutVersion().isNullVersion()) {
// Re-create the .inflight file by opening a new file and write the commit metadata in
createFileInMetaPath(fromInstant.getFileName(), data, false);
Path fromInstantPath = new Path(metaClient.getMetaPath(), fromInstant.getFileName());
Path toInstantPath = new Path(metaClient.getMetaPath(), toInstant.getFileName());
boolean success = metaClient.getFs().rename(fromInstantPath, toInstantPath);
if (!success) {
throw new HoodieIOException("Could not rename " + fromInstantPath + " to " + toInstantPath);
}
} else {
// Ensures old state exists in timeline
LOG.info("Checking for file exists ?" + new Path(metaClient.getMetaPath(), fromInstant.getFileName()));
Preconditions.checkArgument(metaClient.getFs().exists(new Path(metaClient.getMetaPath(),
fromInstant.getFileName())));
// Use Write Once to create Target File
createImmutableFileInPath(new Path(metaClient.getMetaPath(), toInstant.getFileName()), data);
LOG.info("Create new file for toInstant ?" + new Path(metaClient.getMetaPath(), toInstant.getFileName()));
}
} catch (IOException e) {
throw new HoodieIOException("Could not complete " + fromInstant, e);
}
}
private void revertStateTransition(HoodieInstant curr, HoodieInstant revert) {
Preconditions.checkArgument(curr.getTimestamp().equals(revert.getTimestamp()));
Path revertFilePath = new Path(metaClient.getMetaPath(), revert.getFileName());
private void revertCompleteToInflight(HoodieInstant completed, HoodieInstant inflight) {
Preconditions.checkArgument(completed.getTimestamp().equals(inflight.getTimestamp()));
Path inFlightCommitFilePath = new Path(metaClient.getMetaPath(), inflight.getFileName());
Path commitFilePath = new Path(metaClient.getMetaPath(), completed.getFileName());
try {
if (!metaClient.getFs().exists(revertFilePath)) {
Path currFilePath = new Path(metaClient.getMetaPath(), curr.getFileName());
boolean success = metaClient.getFs().rename(currFilePath, revertFilePath);
if (!success) {
throw new HoodieIOException("Could not rename " + currFilePath + " to " + revertFilePath);
if (metaClient.getTimelineLayoutVersion().isNullVersion()) {
if (!metaClient.getFs().exists(inFlightCommitFilePath)) {
boolean success = metaClient.getFs().rename(commitFilePath, inFlightCommitFilePath);
if (!success) {
throw new HoodieIOException(
"Could not rename " + commitFilePath + " to " + inFlightCommitFilePath);
}
}
LOG.info("Renamed " + currFilePath + " to " + revertFilePath);
} else {
Path requestedInstantFilePath = new Path(metaClient.getMetaPath(),
new HoodieInstant(State.REQUESTED, inflight.getAction(), inflight.getTimestamp()).getFileName());
// If inflight and requested files do not exist, create one
if (!metaClient.getFs().exists(requestedInstantFilePath)) {
metaClient.getFs().create(requestedInstantFilePath, false).close();
}
if (!metaClient.getFs().exists(inFlightCommitFilePath)) {
metaClient.getFs().create(inFlightCommitFilePath, false).close();
}
boolean success = metaClient.getFs().delete(commitFilePath, false);
Preconditions.checkArgument(success, "State Reverting failed");
}
} catch (IOException e) {
throw new HoodieIOException("Could not complete revert " + curr, e);
throw new HoodieIOException("Could not complete revert " + completed, e);
}
}
public void saveToInflight(HoodieInstant instant, Option<byte[]> content) {
Preconditions.checkArgument(instant.isInflight());
createFileInMetaPath(instant.getFileName(), content);
public void transitionRequestedToInflight(HoodieInstant requested, Option<byte[]> content) {
HoodieInstant inflight = new HoodieInstant(State.INFLIGHT, requested.getAction(), requested.getTimestamp());
Preconditions.checkArgument(requested.isRequested(), "Instant " + requested + " in wrong state");
transitionState(requested, inflight, content);
}
public void saveToCompactionRequested(HoodieInstant instant, Option<byte[]> content) {
saveToCompactionRequested(instant, content, false);
}
public void saveToCompactionRequested(HoodieInstant instant, Option<byte[]> content, boolean overwrite) {
Preconditions.checkArgument(instant.getAction().equals(HoodieTimeline.COMPACTION_ACTION));
// Write workload to auxiliary folder
createFileInAuxiliaryFolder(instant, content);
createFileInMetaPath(instant.getFileName(), content);
createFileInMetaPath(instant.getFileName(), content, overwrite);
}
public void saveToCleanRequested(HoodieInstant instant, Option<byte[]> content) {
@@ -416,12 +474,16 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
// Write workload to auxiliary folder
createFileInAuxiliaryFolder(instant, content);
// Plan is only stored in auxiliary folder
createFileInMetaPath(instant.getFileName(), Option.empty());
createFileInMetaPath(instant.getFileName(), Option.empty(), false);
}
private void createFileInMetaPath(String filename, Option<byte[]> content) {
private void createFileInMetaPath(String filename, Option<byte[]> content, boolean allowOverwrite) {
Path fullPath = new Path(metaClient.getMetaPath(), filename);
createFileInPath(fullPath, content);
if (allowOverwrite || metaClient.getTimelineLayoutVersion().isNullVersion()) {
createFileInPath(fullPath, content);
} else {
createImmutableFileInPath(fullPath, content);
}
}
private void createFileInPath(Path fullPath, Option<byte[]> content) {
@@ -445,6 +507,32 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
}
}
/**
* Creates a new file in timeline with overwrite set to false. This ensures
* files are created only once and never rewritten
* @param fullPath File Path
* @param content Content to be stored
*/
private void createImmutableFileInPath(Path fullPath, Option<byte[]> content) {
FSDataOutputStream fsout = null;
try {
fsout = metaClient.getFs().create(fullPath, false);
if (content.isPresent()) {
fsout.write(content.get());
}
} catch (IOException e) {
throw new HoodieIOException("Failed to create file " + fullPath, e);
} finally {
try {
if (null != fsout) {
fsout.close();
}
} catch (IOException e) {
throw new HoodieIOException("Failed to close file " + fullPath, e);
}
}
}
private Option<byte[]> readDataFromPath(Path detailPath) {
try (FSDataInputStream is = metaClient.getFs().open(detailPath)) {
return Option.of(FileIOUtils.readAsByteArray(is));

View File

@@ -70,7 +70,6 @@ public class HoodieDefaultTimeline implements HoodieTimeline {
} catch (NoSuchAlgorithmException nse) {
throw new HoodieException(nse);
}
this.timelineHash = StringUtils.toHexString(md.digest());
}
@@ -94,15 +93,15 @@ public class HoodieDefaultTimeline implements HoodieTimeline {
}
@Override
public HoodieTimeline filterInflightsExcludingCompaction() {
public HoodieTimeline filterPendingExcludingCompaction() {
return new HoodieDefaultTimeline(instants.stream().filter(instant -> {
return instant.isInflight() && (!instant.getAction().equals(HoodieTimeline.COMPACTION_ACTION));
return (!instant.isCompleted()) && (!instant.getAction().equals(HoodieTimeline.COMPACTION_ACTION));
}), details);
}
@Override
public HoodieTimeline filterCompletedInstants() {
return new HoodieDefaultTimeline(instants.stream().filter(s -> !s.isInflight()), details);
return new HoodieDefaultTimeline(instants.stream().filter(HoodieInstant::isCompleted), details);
}
@Override
@@ -223,5 +222,4 @@ public class HoodieDefaultTimeline implements HoodieTimeline {
public String toString() {
return this.getClass().getName() + ": " + instants.stream().map(Object::toString).collect(Collectors.joining(","));
}
}

View File

@@ -21,9 +21,12 @@ package org.apache.hudi.common.table.timeline;
import org.apache.hudi.common.table.HoodieTimeline;
import org.apache.hudi.common.util.FSUtils;
import com.google.common.collect.ImmutableMap;
import org.apache.hadoop.fs.FileStatus;
import java.io.Serializable;
import java.util.Comparator;
import java.util.Map;
import java.util.Objects;
/**
@@ -32,7 +35,24 @@ import java.util.Objects;
*
* @see HoodieTimeline
*/
public class HoodieInstant implements Serializable {
public class HoodieInstant implements Serializable, Comparable<HoodieInstant> {
/**
* A COMPACTION action eventually becomes COMMIT when completed. So, when grouping instants
* for state transitions, this needs to be taken into account
*/
private static final Map<String, String> COMPARABLE_ACTIONS = new ImmutableMap.Builder<String, String>()
.put(HoodieTimeline.COMPACTION_ACTION, HoodieTimeline.COMMIT_ACTION).build();
public static final Comparator<HoodieInstant> ACTION_COMPARATOR =
Comparator.<HoodieInstant, String>comparing(instant -> getComparableAction(instant.getAction()));
public static final Comparator<HoodieInstant> COMPARATOR = Comparator.comparing(HoodieInstant::getTimestamp)
.thenComparing(ACTION_COMPARATOR).thenComparing(HoodieInstant::getState);
public static final String getComparableAction(String action) {
return COMPARABLE_ACTIONS.containsKey(action) ? COMPARABLE_ACTIONS.get(action) : action;
}
/**
* Instant State.
@@ -116,7 +136,8 @@ public class HoodieInstant implements Serializable {
public String getFileName() {
if (HoodieTimeline.COMMIT_ACTION.equals(action)) {
return isInflight() ? HoodieTimeline.makeInflightCommitFileName(timestamp)
: HoodieTimeline.makeCommitFileName(timestamp);
: isRequested() ? HoodieTimeline.makeRequestedCommitFileName(timestamp)
: HoodieTimeline.makeCommitFileName(timestamp);
} else if (HoodieTimeline.CLEAN_ACTION.equals(action)) {
return isInflight() ? HoodieTimeline.makeInflightCleanerFileName(timestamp)
: isRequested() ? HoodieTimeline.makeRequestedCleanerFileName(timestamp)
@@ -129,7 +150,8 @@ public class HoodieInstant implements Serializable {
: HoodieTimeline.makeSavePointFileName(timestamp);
} else if (HoodieTimeline.DELTA_COMMIT_ACTION.equals(action)) {
return isInflight() ? HoodieTimeline.makeInflightDeltaFileName(timestamp)
: HoodieTimeline.makeDeltaFileName(timestamp);
: isRequested() ? HoodieTimeline.makeRequestedDeltaFileName(timestamp)
: HoodieTimeline.makeDeltaFileName(timestamp);
} else if (HoodieTimeline.COMPACTION_ACTION.equals(action)) {
if (isInflight()) {
return HoodieTimeline.makeInflightCompactionFileName(timestamp);
@@ -166,6 +188,11 @@ public class HoodieInstant implements Serializable {
return Objects.hash(state, action, timestamp);
}
@Override
public int compareTo(HoodieInstant o) {
return COMPARATOR.compare(this, o);
}
@Override
public String toString() {
return "[" + ((isInflight() || isRequested()) ? "==>" : "") + timestamp + "__" + action + "__" + state + "]";

View File

@@ -31,6 +31,7 @@ import org.apache.hudi.common.model.HoodieLogFile;
import org.apache.hudi.common.table.HoodieTimeline;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.util.AvroUtils;
import org.apache.hudi.common.util.CleanerUtils;
import org.apache.hudi.common.util.CompactionUtils;
import org.apache.hudi.common.util.FSUtils;
import org.apache.hudi.common.util.Option;
@@ -259,8 +260,7 @@ public abstract class IncrementalTimelineSyncFileSystemView extends AbstractTabl
*/
private void addCleanInstant(HoodieTimeline timeline, HoodieInstant instant) throws IOException {
LOG.info("Syncing cleaner instant (" + instant + ")");
HoodieCleanMetadata cleanMetadata =
AvroUtils.deserializeHoodieCleanMetadata(timeline.getInstantDetails(instant).get());
HoodieCleanMetadata cleanMetadata = CleanerUtils.getCleanerMetadata(metaClient, instant);
cleanMetadata.getPartitionMetadata().entrySet().stream().forEach(entry -> {
final String basePath = metaClient.getBasePath();
final String partitionPath = entry.getValue().getPartitionPath();

View File

@@ -20,14 +20,17 @@ package org.apache.hudi.common.util;
import org.apache.hudi.avro.model.HoodieCleanMetadata;
import org.apache.hudi.avro.model.HoodieCleanPartitionMetadata;
import org.apache.hudi.avro.model.HoodieCleanerPlan;
import org.apache.hudi.common.HoodieCleanStat;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.versioning.clean.CleanMetadataMigrator;
import org.apache.hudi.common.versioning.clean.CleanV1MigrationHandler;
import org.apache.hudi.common.versioning.clean.CleanV2MigrationHandler;
import com.google.common.collect.ImmutableMap;
import java.io.IOException;
import java.util.List;
public class CleanerUtils {
@@ -54,9 +57,35 @@ public class CleanerUtils {
HoodieCleanMetadata metadata = new HoodieCleanMetadata(startCleanTime,
durationInMs.orElseGet(() -> -1L), totalDeleted, earliestCommitToRetain,
partitionMetadataBuilder.build(), CLEAN_METADATA_VERSION_1);
partitionMetadataBuilder.build(), CLEAN_METADATA_VERSION_2);
return metadata;
}
/**
* Get Latest Version of Hoodie Cleaner Metadata - Output of cleaner operation.
* @param metaClient Hoodie Table Meta Client
* @param cleanInstant Instant referring to clean action
* @return Latest version of Clean metadata corresponding to clean instant
* @throws IOException
*/
public static HoodieCleanMetadata getCleanerMetadata(HoodieTableMetaClient metaClient, HoodieInstant cleanInstant)
throws IOException {
CleanMetadataMigrator metadataMigrator = new CleanMetadataMigrator(metaClient);
return metadataMigrator.upgradeToLatest(metadata, metadata.getVersion());
HoodieCleanMetadata cleanMetadata = AvroUtils.deserializeHoodieCleanMetadata(
metaClient.getActiveTimeline().readPlanAsBytes(cleanInstant).get());
return metadataMigrator.upgradeToLatest(cleanMetadata, cleanMetadata.getVersion());
}
/**
* Get Cleaner Plan corresponding to a clean instant.
* @param metaClient Hoodie Table Meta Client
* @param cleanInstant Instant referring to clean action
* @return Cleaner plan corresponding to clean instant
* @throws IOException
*/
public static HoodieCleanerPlan getCleanerPlan(HoodieTableMetaClient metaClient, HoodieInstant cleanInstant)
throws IOException {
return AvroUtils.deserializeAvroMetadata(metaClient.getActiveTimeline().readPlanAsBytes(cleanInstant).get(),
HoodieCleanerPlan.class);
}
}

View File

@@ -139,8 +139,9 @@ public class CompactionUtils {
public static HoodieCompactionPlan getCompactionPlan(HoodieTableMetaClient metaClient, String compactionInstant)
throws IOException {
CompactionPlanMigrator migrator = new CompactionPlanMigrator(metaClient);
HoodieCompactionPlan compactionPlan = AvroUtils.deserializeCompactionPlan(metaClient.getActiveTimeline()
.getInstantAuxiliaryDetails(HoodieTimeline.getCompactionRequestedInstant(compactionInstant)).get());
HoodieCompactionPlan compactionPlan = AvroUtils.deserializeCompactionPlan(
metaClient.getActiveTimeline().readPlanAsBytes(
HoodieTimeline.getCompactionRequestedInstant(compactionInstant)).get());
return migrator.upgradeToLatest(compactionPlan, compactionPlan.getVersion());
}