1
0

Take 2: Refactor hoodie-common and create right abstractions for Hoodie Storage V2.0

- Refactored timelines to be a single timeline for all active events and one for archived events. CommitTimeline and other timelines can be inferred by applying a filter on the activeTimelime
- Introduced HoodieInstant to abstract different types of action, commit time and if isInFlight
- Implemented other review comments
This commit is contained in:
Prasanna Rajaperumal
2017-01-18 01:00:36 -08:00
parent 8ee777a9bb
commit ccd8cb2407
46 changed files with 1194 additions and 1106 deletions

View File

@@ -16,12 +16,12 @@
package com.uber.hoodie.common.model;
public enum HoodieStorageType {
public enum HoodieFileFormat {
PARQUET(".parquet");
private final String extension;
HoodieStorageType(String extension) {
HoodieFileFormat(String extension) {
this.extension = extension;
}

View File

@@ -16,9 +16,8 @@
package com.uber.hoodie.common.table;
import com.uber.hoodie.common.model.HoodieStorageType;
import com.uber.hoodie.common.model.HoodieFileFormat;
import com.uber.hoodie.common.model.HoodieTableType;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.exception.HoodieIOException;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
@@ -46,10 +45,10 @@ public class HoodieTableConfig implements Serializable {
public static final String HOODIE_PROPERTIES_FILE = "hoodie.properties";
public static final String HOODIE_TABLE_NAME_PROP_NAME = "hoodie.table.name";
public static final String HOODIE_TABLE_TYPE_PROP_NAME = "hoodie.table.type";
public static final String HOODIE_RO_STORAGE_FORMAT_PROP_NAME =
"hoodie.table.ro.storage.format";
public static final String HOODIE_RO_FILE_FORMAT_PROP_NAME =
"hoodie.table.ro.file.format";
public static final HoodieTableType DEFAULT_TABLE_TYPE = HoodieTableType.COPY_ON_WRITE;
public static final HoodieStorageType DEFAULT_RO_STORAGE_FORMAT = HoodieStorageType.PARQUET;
public static final HoodieFileFormat DEFAULT_RO_FILE_FORMAT = HoodieFileFormat.PARQUET;
private Properties props;
public HoodieTableConfig(FileSystem fs, String metaPath) {
@@ -127,12 +126,12 @@ public class HoodieTableConfig implements Serializable {
/**
* Get the Read Optimized Storage Format
*
* @return HoodieStorageType for the Read Optimized Storage format
* @return HoodieFileFormat for the Read Optimized Storage format
*/
public HoodieStorageType getROStorageFormat() {
if (props.contains(HOODIE_RO_STORAGE_FORMAT_PROP_NAME)) {
return HoodieStorageType.valueOf(props.getProperty(HOODIE_RO_STORAGE_FORMAT_PROP_NAME));
public HoodieFileFormat getROFileFormat() {
if (props.contains(HOODIE_RO_FILE_FORMAT_PROP_NAME)) {
return HoodieFileFormat.valueOf(props.getProperty(HOODIE_RO_FILE_FORMAT_PROP_NAME));
}
return DEFAULT_RO_STORAGE_FORMAT;
return DEFAULT_RO_FILE_FORMAT;
}
}

View File

@@ -17,10 +17,8 @@
package com.uber.hoodie.common.table;
import com.uber.hoodie.common.model.HoodieTableType;
import com.uber.hoodie.common.table.timeline.HoodieActiveCommitTimeline;
import com.uber.hoodie.common.table.timeline.HoodieArchivedCommitTimeline;
import com.uber.hoodie.common.table.timeline.HoodieCleanerTimeline;
import com.uber.hoodie.common.table.timeline.HoodieSavePointTimeline;
import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
import com.uber.hoodie.common.table.timeline.HoodieArchivedTimeline;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.exception.DatasetNotFoundException;
import org.apache.hadoop.fs.FileStatus;
@@ -30,8 +28,13 @@ import org.apache.hadoop.fs.PathFilter;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import java.io.Externalizable;
import java.io.File;
import java.io.IOException;
import java.io.ObjectInput;
import java.io.ObjectInputStream;
import java.io.ObjectOutput;
import java.io.ObjectOutputStream;
import java.io.Serializable;
import java.util.Objects;
import java.util.Properties;
@@ -50,28 +53,22 @@ import java.util.Properties;
public class HoodieTableMetaClient implements Serializable {
private final transient static Logger log = LogManager.getLogger(HoodieTableMetaClient.class);
public static String METAFOLDER_NAME = ".hoodie";
public static String COMMIT_EXTENSION = ".commit";
public static String CLEAN_EXTENSION = ".clean";
public static String SAVEPOINT_EXTENSION = ".savepoint";
public static String INFLIGHT_FILE_SUFFIX = ".inflight";
private String basePath;
private transient FileSystem fs;
private String metaPath;
private HoodieTableType tableType;
private HoodieTableConfig tableConfig;
private HoodieTimeline activeCommitTimeline;
private HoodieTimeline archivedCommitTimeline;
private HoodieTimeline savePointTimeline;
private HoodieTimeline cleanerTimeline;
private HoodieActiveTimeline activeTimeline;
private HoodieArchivedTimeline archivedTimeline;
public HoodieTableMetaClient(FileSystem fs, String basePath) throws DatasetNotFoundException {
// Do not load any timeline by default
this(fs, basePath, false);
}
public HoodieTableMetaClient(FileSystem fs, String basePath,
boolean loadActiveCommitTimelineOnLoad) throws DatasetNotFoundException {
public HoodieTableMetaClient(FileSystem fs, String basePath, boolean loadActiveTimelineOnLoad)
throws DatasetNotFoundException {
log.info("Loading HoodieTableMetaClient from " + basePath);
this.basePath = basePath;
this.fs = fs;
@@ -82,14 +79,15 @@ public class HoodieTableMetaClient implements Serializable {
this.tableConfig = new HoodieTableConfig(fs, metaPath);
this.tableType = tableConfig.getTableType();
log.info("Finished Loading Table of type " + tableType + " from " + basePath);
if (loadActiveCommitTimelineOnLoad) {
if (loadActiveTimelineOnLoad) {
log.info("Loading Active commit timeline for " + basePath);
getActiveCommitTimeline();
getActiveTimeline();
}
}
/**
* For serailizing and de-serializing
*
* @deprecated
*/
public HoodieTableMetaClient() {
@@ -97,6 +95,7 @@ public class HoodieTableMetaClient implements Serializable {
/**
* This method is only used when this object is deserialized in a spark executor.
*
* @deprecated
*/
private void readObject(java.io.ObjectInputStream in)
@@ -105,6 +104,11 @@ public class HoodieTableMetaClient implements Serializable {
this.fs = FSUtils.getFs();
}
private void writeObject(java.io.ObjectOutputStream out)
throws IOException {
out.defaultWriteObject();
}
/**
* @return Base path
*/
@@ -134,16 +138,16 @@ public class HoodieTableMetaClient implements Serializable {
}
/**
* Get the active commits as a timeline
* Get the active instants as a timeline
*
* @return Active commit timeline
* @return Active instants timeline
* @throws IOException
*/
public synchronized HoodieTimeline getActiveCommitTimeline() {
if (activeCommitTimeline == null) {
activeCommitTimeline = new HoodieActiveCommitTimeline(fs, metaPath);
public synchronized HoodieActiveTimeline getActiveTimeline() {
if (activeTimeline == null) {
activeTimeline = new HoodieActiveTimeline(fs, metaPath);
}
return activeCommitTimeline;
return activeTimeline;
}
/**
@@ -153,40 +157,13 @@ public class HoodieTableMetaClient implements Serializable {
* @return Active commit timeline
* @throws IOException
*/
public HoodieTimeline getArchivedCommitTimeline() {
if (archivedCommitTimeline == null) {
archivedCommitTimeline = new HoodieArchivedCommitTimeline(fs, metaPath);
public synchronized HoodieArchivedTimeline getArchivedTimeline() {
if (archivedTimeline == null) {
archivedTimeline = new HoodieArchivedTimeline(fs, metaPath);
}
return archivedCommitTimeline;
return archivedTimeline;
}
/**
* Get the save points as a timeline.
*
* @return Savepoint timeline
* @throws IOException
*/
public HoodieTimeline getSavePointsTimeline() {
if (savePointTimeline == null) {
savePointTimeline = new HoodieSavePointTimeline(fs, metaPath);
}
return savePointTimeline;
}
/**
* Get the cleaner activity as a timeline.
*
* @return Cleaner activity
* @throws IOException
*/
public HoodieTimeline getCleanerTimeline() {
if (cleanerTimeline == null) {
cleanerTimeline = new HoodieCleanerTimeline(fs, metaPath);
}
return cleanerTimeline;
}
/**
* Helper method to initialize a given path as a hoodie dataset with configs passed in as as Properties
*
@@ -215,34 +192,6 @@ public class HoodieTableMetaClient implements Serializable {
}
// HELPER METHODS TO CREATE META FILE NAMES
public static String makeCommitFileName(String commitTime) {
return commitTime + COMMIT_EXTENSION;
}
public static String makeInflightCommitFileName(String commitTime) {
return commitTime + INFLIGHT_FILE_SUFFIX;
}
public static String makeCleanerFileName(String instant) {
return instant + CLEAN_EXTENSION;
}
public static String makeInflightCleanerFileName(String instant) {
return instant + CLEAN_EXTENSION + INFLIGHT_FILE_SUFFIX;
}
public static String makeInflightSavePointFileName(String commitTime) {
return commitTime + SAVEPOINT_EXTENSION + INFLIGHT_FILE_SUFFIX;
}
public static String makeSavePointFileName(String commitTime) {
return commitTime + SAVEPOINT_EXTENSION;
}
public static String getCommitFromCommitFile(String commitFileName) {
return commitFileName.split("\\.")[0];
}
public static FileStatus[] scanFiles(FileSystem fs, Path metaPath, PathFilter nameFilter)
throws IOException {
return fs.listStatus(metaPath, nameFilter);

View File

@@ -17,153 +17,125 @@
package com.uber.hoodie.common.table;
import com.uber.hoodie.common.table.timeline.HoodieDefaultTimeline;
import com.uber.hoodie.common.table.timeline.HoodieInstant;
import com.uber.hoodie.common.util.FSUtils;
import java.io.IOException;
import java.io.Serializable;
import java.util.Optional;
import java.util.function.BiPredicate;
import java.util.stream.Stream;
/**
* HoodieTimeline allows representation of meta-data events as a timeline.
* Instants are specific points in time represented as strings.
* in this format YYYYMMDDHHmmSS. e.g. 20170101193218
* Any operation on the timeline starts with the inflight instant and then when complete marks
* the completed instant and removes the inflight instant.
* Completed instants are plainly referred to as just instants
* HoodieTimeline is a view of meta-data instants in the hoodie dataset.
* Instants are specific points in time represented as HoodieInstant.
* <p>
* Timelines as immutable once created. Any operation to change the timeline (like create/delete instants)
* will not be reflected unless explicitly reloaded using the reload()
* Timelines are immutable once created and operations create new instance of
* timelines which filter on the instants and this can be chained.
*
* @see com.uber.hoodie.common.table.HoodieTableMetaClient
* @see HoodieDefaultTimeline
* @see HoodieInstant
* @since 0.3.0
*/
public interface HoodieTimeline extends Serializable {
String COMMIT_ACTION = "commit";
String CLEAN_ACTION = "clean";
String SAVEPOINT_ACTION = "savepoint";
String INFLIGHT_EXTENSION = ".inflight";
String COMMIT_EXTENSION = "." + COMMIT_ACTION;
String CLEAN_EXTENSION = "." + CLEAN_ACTION;
String SAVEPOINT_EXTENSION = "." + SAVEPOINT_ACTION;
//this is to preserve backwards compatibility on commit in-flight filenames
String INFLIGHT_COMMIT_EXTENSION = INFLIGHT_EXTENSION;
String INFLIGHT_CLEAN_EXTENSION = "." + CLEAN_ACTION + INFLIGHT_EXTENSION;
String INFLIGHT_SAVEPOINT_EXTENSION = "." + SAVEPOINT_ACTION + INFLIGHT_EXTENSION;
/**
* Find all the completed instants after startTs and before or on endTs
* Filter this timeline to just include the in-flights
*
* @return New instance of HoodieTimeline with just in-flights
*/
HoodieTimeline filterInflights();
/**
* Filter this timeline to just include the completed instants
*
* @return New instance of HoodieTimeline with just completed instants
*/
HoodieTimeline filterCompletedInstants();
/**
* Create a new Timeline with instants after startTs and before or on endTs
*
* @param startTs
* @param endTs
* @return Stream of instants
*/
Stream<String> findInstantsInRange(String startTs, String endTs);
HoodieTimeline findInstantsInRange(String startTs, String endTs);
/**
* Find all the completed instants after startTs
* Create a new Timeline with all the instants after startTs
*
* @param commitTime
* @param numCommits
* @return Stream of instants
*/
Stream<String> findInstantsAfter(String commitTime, int numCommits);
HoodieTimeline findInstantsAfter(String commitTime, int numCommits);
/**
* If the timeline has any completed instants
* If the timeline has any instants
*
* @return true if timeline is not empty
* @return true if timeline is empty
*/
boolean hasInstants();
/**
* If the timeline has any in-complete instants
*
* @return true if timeline has any in-complete instants
*/
boolean hasInflightInstants();
boolean empty();
/**
* @return total number of completed instants
*/
int getTotalInstants();
int countInstants();
/**
* @return first completed instant if available
*/
Optional<String> firstInstant();
Optional<HoodieInstant> firstInstant();
/**
* @param n
* @return nth completed instant from the first completed instant
*/
Optional<String> nthInstant(int n);
Optional<HoodieInstant> nthInstant(int n);
/**
* @return last completed instant if available
*/
Optional<String> lastInstant();
Optional<HoodieInstant> lastInstant();
/**
* @param n
* @return nth completed instant going back from the last completed instant
*/
Optional<String> nthFromLastInstant(int n);
Optional<HoodieInstant> nthFromLastInstant(int n);
/**
* @return true if the passed instant is present as a completed instant on the timeline
*/
boolean containsInstant(String instant);
boolean containsInstant(HoodieInstant instant);
/**
* @return true if the passed instant is present as a completed instant on the timeline or
* if the instant is before the first completed instant in the timeline
*/
boolean containsOrBeforeTimelineStarts(String instant);
boolean containsOrBeforeTimelineStarts(String ts);
/**
* @return Get the stream of completed instants
*/
Stream<String> getInstants();
/**
* @return Get the stream of in-flight instants
*/
Stream<String> getInflightInstants();
Stream<HoodieInstant> getInstants();
/**
* @return true if the passed in instant is before the first completed instant in the timeline
*/
boolean isInstantBeforeTimelineStarts(String instant);
/**
* Register the passed in instant as a in-flight
*
* @param instant
*/
void saveInstantAsInflight(String instant);
/**
* Register the passed in instant as a completed instant.
* It needs to have a corresponding in-flight instant, otherwise it will fail.
* Pass a optional byte[] to save with the instant.
*
* @param instant
* @param data
*/
void saveInstantAsComplete(String instant, Optional<byte[]> data);
/**
* Un-Register a completed instant as in-flight. This is usually atomic way to
* revert the effects of a operation on hoodie datasets
*
* @param instant
*/
void revertInstantToInflight(String instant);
/**
* Remove the in-flight instant from the timeline
*
* @param instant
*/
void removeInflightFromTimeline(String instant);
/**
* Reload the timeline. Timelines are immutable once created.
*
* @return
* @throws IOException
*/
HoodieTimeline reload() throws IOException;
boolean isBeforeTimelineStarts(String ts);
/**
* Read the completed instant details
@@ -171,7 +143,7 @@ public interface HoodieTimeline extends Serializable {
* @param instant
* @return
*/
Optional<byte[]> readInstantDetails(String instant);
Optional<byte[]> getInstantDetails(HoodieInstant instant);
/**
* Helper methods to compare instants
@@ -183,8 +155,55 @@ public interface HoodieTimeline extends Serializable {
(commit1, commit2) -> commit1.compareTo(commit2) <= 0;
BiPredicate<String, String> LESSER = (commit1, commit2) -> commit1.compareTo(commit2) < 0;
default boolean compareInstants(String commit1, String commit2,
default boolean compareTimestamps(String commit1, String commit2,
BiPredicate<String, String> predicateToApply) {
return predicateToApply.test(commit1, commit2);
}
static HoodieInstant getCompletedInstant(final HoodieInstant instant) {
return new HoodieInstant(false, instant.getAction(), instant.getTimestamp());
}
static HoodieInstant getInflightInstant(final HoodieInstant instant) {
return new HoodieInstant(true, instant.getAction(), instant.getTimestamp());
}
static String makeCommitFileName(String commitTime) {
return commitTime + HoodieTimeline.COMMIT_EXTENSION;
}
static String makeInflightCommitFileName(String commitTime) {
return commitTime + HoodieTimeline.INFLIGHT_COMMIT_EXTENSION;
}
static String makeCleanerFileName(String instant) {
return instant + HoodieTimeline.CLEAN_EXTENSION;
}
static String makeInflightCleanerFileName(String instant) {
return instant + HoodieTimeline.INFLIGHT_CLEAN_EXTENSION;
}
static String makeInflightSavePointFileName(String commitTime) {
return commitTime + HoodieTimeline.INFLIGHT_SAVEPOINT_EXTENSION;
}
static String makeSavePointFileName(String commitTime) {
return commitTime + HoodieTimeline.SAVEPOINT_EXTENSION;
}
static String getCommitFromCommitFile(String commitFileName) {
return commitFileName.split("\\.")[0];
}
static String makeFileNameAsComplete(String fileName) {
return fileName.replace(HoodieTimeline.INFLIGHT_EXTENSION, "");
}
static String makeFileNameAsInflight(String fileName) {
return fileName + HoodieTimeline.INFLIGHT_EXTENSION;
}
}

View File

@@ -51,7 +51,7 @@ public interface TableFileSystemView {
* @param maxCommitTime
* @return
*/
Stream<HoodieDataFile> streamLatestVersionInPartition(String partitionPathStr,
Stream<HoodieDataFile> getLatestVersionInPartition(String partitionPathStr,
String maxCommitTime);
/**
@@ -60,7 +60,7 @@ public interface TableFileSystemView {
* @param partitionPath
* @return
*/
Stream<List<HoodieDataFile>> streamEveryVersionInPartition(String partitionPath);
Stream<List<HoodieDataFile>> getEveryVersionInPartition(String partitionPath);
/**
* Stream all the versions from the passed in fileStatus[] with commit times containing in commitsToReturn.
@@ -69,7 +69,7 @@ public interface TableFileSystemView {
* @param commitsToReturn
* @return
*/
Stream<HoodieDataFile> streamLatestVersionInRange(FileStatus[] fileStatuses,
Stream<HoodieDataFile> getLatestVersionInRange(FileStatus[] fileStatuses,
List<String> commitsToReturn);
/**
@@ -79,15 +79,15 @@ public interface TableFileSystemView {
* @param maxCommitToReturn
* @return
*/
Stream<HoodieDataFile> streamLatestVersionsBeforeOrOn(FileStatus[] fileStatuses,
Stream<HoodieDataFile> getLatestVersionsBeforeOrOn(FileStatus[] fileStatuses,
String maxCommitToReturn);
/**
* Stream latest versions from the passed in FileStatus[].
* Similar to calling streamLatestVersionsBeforeOrOn(fileStatuses, currentTimeAsCommitTime)
* Similar to calling getLatestVersionsBeforeOrOn(fileStatuses, currentTimeAsCommitTime)
*
* @param fileStatuses
* @return
*/
Stream<HoodieDataFile> streamLatestVersions(FileStatus[] fileStatuses);
Stream<HoodieDataFile> getLatestVersions(FileStatus[] fileStatuses);
}

View File

@@ -1,77 +0,0 @@
/*
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
*
* Licensed 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 com.uber.hoodie.common.table.timeline;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.exception.HoodieIOException;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import java.io.IOException;
import java.util.Arrays;
import java.util.stream.Collectors;
/**
* Active commit timeline. Much optimized for reading than the ArchivedTimeline.
*/
public class HoodieActiveCommitTimeline extends HoodieDefaultTimeline {
public HoodieActiveCommitTimeline(FileSystem fs, String metaPath) {
super(fs, metaPath);
String completedInstantExtension = HoodieTableMetaClient.COMMIT_EXTENSION;
String inflightInstantExtension = INFLIGHT_EXTENSION;
FileStatus[] fileStatuses;
try {
fileStatuses = HoodieTableMetaClient.scanFiles(fs, new Path(metaPath),
path -> path.toString().endsWith(completedInstantExtension) || path.toString()
.endsWith(inflightInstantExtension));
} catch (IOException e) {
throw new HoodieIOException("Failed to scan metadata", e);
}
this.instants = Arrays.stream(fileStatuses)
.filter(status -> status.getPath().getName().endsWith(completedInstantExtension))
.map(fileStatus -> fileStatus.getPath().getName().replaceAll(completedInstantExtension, ""))
.sorted().collect(Collectors.toList());
this.inflights = Arrays.stream(fileStatuses).filter(
status -> status.getPath().getName().endsWith(inflightInstantExtension)).map(
fileStatus -> fileStatus.getPath().getName()
.replaceAll(inflightInstantExtension, "")).sorted()
.collect(Collectors.toList());
}
@Override
public String getInflightFileName(String instant) {
return HoodieTableMetaClient.makeInflightCommitFileName(instant);
}
@Override
public String getCompletedFileName(String instant) {
return HoodieTableMetaClient.makeCommitFileName(instant);
}
@Override
protected String getTimelineName() {
return "commit";
}
@Override
public HoodieTimeline reload() throws IOException {
return new HoodieActiveCommitTimeline(fs, metaPath);
}
}

View File

@@ -0,0 +1,252 @@
/*
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
*
* Licensed 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 com.uber.hoodie.common.table.timeline;
import com.google.common.io.Closeables;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.exception.HoodieIOException;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
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.Arrays;
import java.util.Comparator;
import java.util.Optional;
import java.util.function.Function;
import java.util.function.Predicate;
import java.util.stream.Collectors;
import java.util.stream.Stream;
/**
* Represents the Active Timeline for the HoodieDataset. Instants for the last 12 hours (configurable)
* is in the ActiveTimeline and the rest are Archived. ActiveTimeline is a special timeline
* that allows for creation of instants on the timeline.
* <p></p>
* The timeline is not automatically reloaded on any mutation operation, clients have to manually call reload()
* so that they can chain multiple mutations to the timeline and then call reload() once.
* <p></p>
* This class can be serialized and de-serialized and on de-serialization the FileSystem is re-initialized.
*/
public class HoodieActiveTimeline extends HoodieDefaultTimeline {
private final transient static Logger log = LogManager.getLogger(HoodieActiveTimeline.class);
private String metaPath;
private transient FileSystem fs;
protected HoodieActiveTimeline(FileSystem fs, String metaPath, String[] includedExtensions) {
// Filter all the filter in the metapath and include only the extensions passed and
// convert them into HoodieInstant
try {
this.instants =
Arrays.stream(HoodieTableMetaClient.scanFiles(fs, new Path(metaPath), path -> {
// Include only the meta files with extensions that needs to be included
String extension = FSUtils.getFileExtension(path.getName());
return Arrays.stream(includedExtensions).anyMatch(Predicate.isEqual(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());
log.info("Loaded instants " + instants);
} catch (IOException e) {
throw new HoodieIOException("Failed to scan metadata", e);
}
this.fs = fs;
this.metaPath = metaPath;
// 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, Optional<byte[]>> & Serializable) this::getInstantDetails;
}
public HoodieActiveTimeline(FileSystem fs, String metaPath) {
this(fs, metaPath,
new String[] {COMMIT_EXTENSION, INFLIGHT_COMMIT_EXTENSION, SAVEPOINT_EXTENSION,
INFLIGHT_SAVEPOINT_EXTENSION, CLEAN_EXTENSION, INFLIGHT_CLEAN_EXTENSION});
}
/**
* For serialization and de-serialization only.
* @deprecated
*/
public HoodieActiveTimeline() {
}
/**
* This method is only used when this object is deserialized in a spark executor.
*
* @deprecated
*/
private void readObject(java.io.ObjectInputStream in)
throws IOException, ClassNotFoundException {
in.defaultReadObject();
this.fs = FSUtils.getFs();
}
/**
* Get only the commits (inflight and completed) in the active timeline
*
* @return
*/
public HoodieTimeline getCommitTimeline() {
return new HoodieDefaultTimeline(filterInstantsByAction(COMMIT_ACTION),
(Function<HoodieInstant, Optional<byte[]>> & Serializable) this::getInstantDetails);
}
/**
* Get only the cleaner action (inflight and completed) in the active timeline
*
* @return
*/
public HoodieTimeline getCleanerTimeline() {
return new HoodieDefaultTimeline(filterInstantsByAction(CLEAN_ACTION),
(Function<HoodieInstant, Optional<byte[]>> & Serializable) this::getInstantDetails);
}
/**
* Get only the save point action (inflight and completed) in the active timeline
*
* @return
*/
public HoodieTimeline getSavePointTimeline() {
return new HoodieDefaultTimeline(filterInstantsByAction(SAVEPOINT_ACTION),
(Function<HoodieInstant, Optional<byte[]>> & Serializable) this::getInstantDetails);
}
protected Stream<HoodieInstant> filterInstantsByAction(String action) {
return instants.stream().filter(s -> s.getAction().equals(action));
}
public void createInflight(HoodieInstant instant) {
log.info("Creating a new in-flight instant " + instant);
// Create the in-flight file
createFileInMetaPath(instant.getFileName(), Optional.empty());
}
public void saveAsComplete(HoodieInstant instant, Optional<byte[]> data) {
log.info("Marking instant complete " + instant);
moveInflightToComplete(instant, HoodieTimeline.getCompletedInstant(instant), data);
log.info("Completed " + instant);
}
public void revertToInflight(HoodieInstant instant) {
log.info("Reverting instant to inflight " + instant);
moveCompleteToInflight(instant, HoodieTimeline.getInflightInstant(instant));
log.info("Reverted " + instant + " to inflight");
}
public void deleteInflight(HoodieInstant instant) {
log.info("Deleting in-flight " + instant);
Path inFlightCommitFilePath = new Path(metaPath, instant.getFileName());
try {
boolean result = fs.delete(inFlightCommitFilePath, false);
if (result) {
log.info("Removed in-flight " + instant);
} else {
throw new HoodieIOException("Could not delete in-flight instant " + instant);
}
} catch (IOException e) {
throw new HoodieIOException(
"Could not remove inflight commit " + inFlightCommitFilePath, e);
}
}
@Override
public Optional<byte[]> getInstantDetails(HoodieInstant instant) {
Path detailPath = new Path(metaPath, instant.getFileName());
return readDataFromPath(detailPath);
}
protected void moveInflightToComplete(HoodieInstant inflight, HoodieInstant completed,
Optional<byte[]> data) {
Path commitFilePath = new Path(metaPath, completed.getFileName());
try {
// open a new file and write the commit metadata in
Path inflightCommitFile = new Path(metaPath, inflight.getFileName());
createFileInMetaPath(inflight.getFileName(), data);
boolean success = fs.rename(inflightCommitFile, commitFilePath);
if (!success) {
throw new HoodieIOException(
"Could not rename " + inflightCommitFile + " to " + commitFilePath);
}
} catch (IOException e) {
throw new HoodieIOException("Could not complete " + inflight, e);
}
}
protected void moveCompleteToInflight(HoodieInstant completed, HoodieInstant inflight) {
Path inFlightCommitFilePath = new Path(metaPath, inflight.getFileName());
try {
if (!fs.exists(inFlightCommitFilePath)) {
Path commitFilePath = new Path(metaPath, completed.getFileName());
boolean success = fs.rename(commitFilePath, inFlightCommitFilePath);
if (!success) {
throw new HoodieIOException(
"Could not rename " + commitFilePath + " to " + inFlightCommitFilePath);
}
}
} catch (IOException e) {
throw new HoodieIOException("Could not complete revert " + completed, e);
}
}
protected void createFileInMetaPath(String filename, Optional<byte[]> content) {
Path fullPath = new Path(metaPath, filename);
try {
if (!content.isPresent()) {
if (fs.createNewFile(fullPath)) {
log.info("Created a new file in meta path: " + fullPath);
return;
}
} else {
FSDataOutputStream fsout = fs.create(fullPath, true);
fsout.write(content.get());
fsout.close();
return;
}
throw new HoodieIOException("Failed to create file " + fullPath);
} catch (IOException e) {
throw new HoodieIOException("Failed to create file " + fullPath, e);
}
}
protected Optional<byte[]> readDataFromPath(Path detailPath) {
FSDataInputStream is = null;
try {
is = fs.open(detailPath);
return Optional.of(IOUtils.toByteArray(is));
} catch (IOException e) {
throw new HoodieIOException("Could not read commit details from " + detailPath, e);
} finally {
if (is != null) {
Closeables.closeQuietly(is);
}
}
}
public HoodieActiveTimeline reload() {
return new HoodieActiveTimeline(fs, metaPath);
}
}

View File

@@ -18,30 +18,42 @@ package com.uber.hoodie.common.table.timeline;
import com.google.common.io.Closeables;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.exception.HoodieIOException;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.SequenceFile;
import org.apache.hadoop.io.Text;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import java.io.IOException;
import java.util.ArrayList;
import java.io.Serializable;
import java.util.Arrays;
import java.util.HashMap;
import java.util.Map;
import java.util.Optional;
import java.util.function.Function;
import java.util.stream.Collectors;
/**
* Archived commit timeline. These commits are usually cleaned up and the meta data is archived for
* future triaging
*
* @since 0.3.0
* Represents the Archived Timeline for the HoodieDataset. Instants for the last 12 hours (configurable)
* is in the ActiveTimeline and the rest are in ArchivedTimeline.
* <p></p>
* Instants are read from the archive file during initialization and never refreshed. To refresh, clients
* need to call reload()
* <p></p>
* This class can be serialized and de-serialized and on de-serialization the FileSystem is re-initialized.
*/
public class HoodieArchivedCommitTimeline extends HoodieDefaultTimeline {
public class HoodieArchivedTimeline extends HoodieDefaultTimeline {
private static final String HOODIE_COMMIT_ARCHIVE_LOG_FILE = "commits.archived";
private transient Map<String, byte[]> readCommits = new HashMap<>();
private transient FileSystem fs;
private String metaPath;
private Map<String, byte[]> readCommits = new HashMap<>();
public HoodieArchivedCommitTimeline(FileSystem fs, String metaPath) {
private final transient static Logger log = LogManager.getLogger(HoodieArchivedTimeline.class);
public HoodieArchivedTimeline(FileSystem fs, String metaPath) {
// Read back the commits to make sure
Path archiveLogPath = getArchiveLogPath(metaPath);
try {
@@ -55,8 +67,9 @@ public class HoodieArchivedCommitTimeline extends HoodieDefaultTimeline {
// This is okay because only tooling will load the archived commit timeline today
readCommits.put(key.toString(), Arrays.copyOf(val.getBytes(), val.getLength()));
}
this.instants = new ArrayList<>(readCommits.keySet());
this.inflights = new ArrayList<>(0);
this.instants = readCommits.keySet().stream().map(
s -> new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, s)).collect(
Collectors.toList());
} finally {
Closeables.closeQuietly(reader);
}
@@ -64,58 +77,42 @@ public class HoodieArchivedCommitTimeline extends HoodieDefaultTimeline {
throw new HoodieIOException(
"Could not load archived commit timeline from path " + archiveLogPath, e);
}
// 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, Optional<byte[]>> & Serializable) this::getInstantDetails;
this.fs = fs;
this.metaPath = metaPath;
}
@Override
public void saveInstantAsInflight(String instant) {
throw new UnsupportedOperationException(
"Could not save inflight instant in ArchivedTimeline " + instant);
/**
* For serialization and de-serialization only.
* @deprecated
*/
public HoodieArchivedTimeline() {
}
@Override
public void saveInstantAsComplete(String instant, Optional<byte[]> data) {
throw new UnsupportedOperationException(
"Could not save instant as complete in ArchivedTimeline " + instant);
/**
* This method is only used when this object is deserialized in a spark executor.
*
* @deprecated
*/
private void readObject(java.io.ObjectInputStream in)
throws IOException, ClassNotFoundException {
in.defaultReadObject();
this.fs = FSUtils.getFs();
}
@Override
public void revertInstantToInflight(String instant) {
throw new UnsupportedOperationException(
"Could not revert instant in ArchivedTimeline " + instant);
}
@Override
public void removeInflightFromTimeline(String instant) {
throw new UnsupportedOperationException(
"Could not delete inflight instant from ArchivedTimeline " + instant);
}
@Override
public HoodieTimeline reload() throws IOException {
return new HoodieArchivedCommitTimeline(fs, metaPath);
}
@Override
public Optional<byte[]> readInstantDetails(String instant) {
return Optional.ofNullable(readCommits.get(instant));
}
@Override
protected String getInflightFileName(String instant) {
throw new UnsupportedOperationException("No inflight filename for archived commits");
}
@Override
protected String getCompletedFileName(String instant) {
throw new UnsupportedOperationException("No inflight filename for archived commits");
}
@Override
protected String getTimelineName() {
return "archived-commits";
}
public static Path getArchiveLogPath(String metaPath) {
return new Path(metaPath, HOODIE_COMMIT_ARCHIVE_LOG_FILE);
}
@Override
public Optional<byte[]> getInstantDetails(HoodieInstant instant) {
return Optional.ofNullable(readCommits.get(instant.getTimestamp()));
}
public HoodieArchivedTimeline reload() {
return new HoodieArchivedTimeline(fs, metaPath);
}
}

View File

@@ -1,57 +0,0 @@
/*
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
*
* Licensed 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 com.uber.hoodie.common.table.timeline;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import java.io.IOException;
import java.util.Optional;
public class HoodieCleanerTimeline extends HoodieDefaultTimeline {
public HoodieCleanerTimeline(FileSystem fs, String path) {
super(fs, path, HoodieTableMetaClient.CLEAN_EXTENSION);
}
@Override
public HoodieTimeline reload() throws IOException {
return new HoodieCleanerTimeline(fs, metaPath);
}
@Override
public Optional<byte[]> readInstantDetails(String instant) {
// TODO - Nothing about the clean written today - this should change
return Optional.empty();
}
@Override
protected String getInflightFileName(String instant) {
return HoodieTableMetaClient.makeInflightCleanerFileName(instant);
}
@Override
protected String getCompletedFileName(String instant) {
return HoodieTableMetaClient.makeCleanerFileName(instant);
}
@Override
protected String getTimelineName() {
return "cleaner";
}
}

View File

@@ -16,329 +16,131 @@
package com.uber.hoodie.common.table.timeline;
import com.google.common.io.Closeables;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.exception.HoodieIOException;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import java.io.IOException;
import java.util.Arrays;
import java.util.List;
import java.util.Optional;
import java.util.function.Function;
import java.util.stream.Collectors;
import java.util.stream.Stream;
/**
* HoodieTimeline allows representation of meta-data events as a timeline.
* Instants are specific points in time represented as strings.
* in this format YYYYMMDDHHmmSS. e.g. 20170101193218
* Any operation on the timeline starts with the inflight instant and then when complete marks
* the completed instant and removes the inflight instant.
* Completed instants are plainly referred to as just instants
* <p>
* Timelines as immutable once created. Any operation to change the timeline (like create/delete instants)
* will not be reflected unless explicitly reloaded using the reload()
* HoodieDefaultTimeline is a default implementation of the HoodieTimeline.
* It provides methods to inspect a List[HoodieInstant]. Function to get the details of the instant
* is passed in as a lamdba.
*
* @see com.uber.hoodie.common.table.HoodieTableMetaClient
* @see HoodieTimeline
* @since 0.3.0
*/
public abstract class HoodieDefaultTimeline implements HoodieTimeline {
public class HoodieDefaultTimeline implements HoodieTimeline {
private final transient static Logger log = LogManager.getLogger(HoodieDefaultTimeline.class);
public static final String INFLIGHT_EXTENSION = ".inflight";
protected String metaPath;
protected transient FileSystem fs;
protected List<String> inflights;
protected List<String> instants;
protected Function<HoodieInstant, Optional<byte[]>> details;
protected List<HoodieInstant> instants;
public HoodieDefaultTimeline(FileSystem fs, String metaPath, String fileExtension) {
String completedInstantExtension = fileExtension;
String inflightInstantExtension = fileExtension + INFLIGHT_EXTENSION;
FileStatus[] fileStatuses;
try {
fileStatuses = HoodieTableMetaClient.scanFiles(fs, new Path(metaPath),
path -> path.toString().endsWith(completedInstantExtension) || path.toString()
.endsWith(inflightInstantExtension));
} catch (IOException e) {
throw new HoodieIOException("Failed to scan metadata", e);
}
this.instants = Arrays.stream(fileStatuses)
.filter(status -> status.getPath().getName().endsWith(completedInstantExtension))
.map(fileStatus -> fileStatus.getPath().getName().replaceAll(completedInstantExtension, ""))
.sorted().collect(Collectors.toList());
this.inflights = Arrays.stream(fileStatuses).filter(
status -> status.getPath().getName().endsWith(inflightInstantExtension)).map(
fileStatus -> fileStatus.getPath().getName()
.replaceAll(inflightInstantExtension, "")).sorted()
.collect(Collectors.toList());
this.fs = fs;
this.metaPath = metaPath;
}
public HoodieDefaultTimeline(Stream<String> instants, Stream<String> inflights) {
public HoodieDefaultTimeline(Stream<HoodieInstant> instants,
Function<HoodieInstant, Optional<byte[]>> details) {
this.instants = instants.collect(Collectors.toList());
this.inflights = inflights.collect(Collectors.toList());
}
/**
* This constructor only supports backwards compatibility in inflight commits in ActiveCommitTimeline.
* This should never be used.
*
* @param fs
* @param metaPath
* @deprecated
*/
public HoodieDefaultTimeline(FileSystem fs, String metaPath) {
this.fs = fs;
this.metaPath = metaPath;
this.details = details;
}
/**
* For serailizing and de-serializing
*
* @deprecated
*/
public HoodieDefaultTimeline() {
}
public HoodieTimeline filterInflights() {
return new HoodieDefaultTimeline(instants.stream().filter(HoodieInstant::isInflight),
details);
}
/**
* This method is only used when this object is deserialized in a spark executor.
* @deprecated
*/
private void readObject(java.io.ObjectInputStream in)
throws IOException, ClassNotFoundException {
in.defaultReadObject();
this.fs = FSUtils.getFs();
public HoodieTimeline filterCompletedInstants() {
return new HoodieDefaultTimeline(instants.stream().filter(s -> !s.isInflight()), details);
}
@Override
public Stream<String> findInstantsInRange(String startTs, String endTs) {
return instants.stream().filter(
s -> compareInstants(s, startTs, GREATER) && compareInstants(s, endTs,
LESSER_OR_EQUAL));
public HoodieDefaultTimeline findInstantsInRange(String startTs, String endTs) {
return new HoodieDefaultTimeline(instants.stream().filter(
s -> compareTimestamps(s.getTimestamp(), startTs, GREATER) && compareTimestamps(
s.getTimestamp(), endTs, LESSER_OR_EQUAL)), details);
}
@Override
public Stream<String> findInstantsAfter(String commitTime, int numCommits) {
return instants.stream().filter(s -> compareInstants(s, commitTime, GREATER))
.limit(numCommits);
public HoodieDefaultTimeline findInstantsAfter(String commitTime, int numCommits) {
return new HoodieDefaultTimeline(
instants.stream().filter(s -> compareTimestamps(s.getTimestamp(), commitTime, GREATER))
.limit(numCommits), details);
}
@Override
public boolean hasInstants() {
return instants.stream().count() != 0;
public boolean empty() {
return !instants.stream().findFirst().isPresent();
}
@Override
public boolean hasInflightInstants() {
return inflights.stream().count() != 0;
}
@Override
public int getTotalInstants() {
public int countInstants() {
return new Long(instants.stream().count()).intValue();
}
@Override
public Optional<String> firstInstant() {
public Optional<HoodieInstant> firstInstant() {
return instants.stream().findFirst();
}
@Override
public Optional<String> nthInstant(int n) {
if(!hasInstants() || n >= getTotalInstants()) {
public Optional<HoodieInstant> nthInstant(int n) {
if (empty() || n >= countInstants()) {
return Optional.empty();
}
return Optional.of(instants.get(n));
}
@Override
public Optional<String> lastInstant() {
return hasInstants() ? nthInstant(getTotalInstants() - 1) : Optional.empty();
public Optional<HoodieInstant> lastInstant() {
return empty() ? Optional.empty() : nthInstant(countInstants() - 1);
}
@Override
public Optional<String> nthFromLastInstant(int n) {
if(getTotalInstants() < n + 1) {
public Optional<HoodieInstant> nthFromLastInstant(int n) {
if (countInstants() < n + 1) {
return Optional.empty();
}
return nthInstant(getTotalInstants() - 1 - n);
return nthInstant(countInstants() - 1 - n);
}
@Override
public boolean containsInstant(String instant) {
public boolean containsInstant(HoodieInstant instant) {
return instants.stream().anyMatch(s -> s.equals(instant));
}
@Override
public boolean containsOrBeforeTimelineStarts(String instant) {
return containsInstant(instant) || isInstantBeforeTimelineStarts(instant);
return instants.stream().anyMatch(s -> s.getTimestamp().equals(instant))
|| isBeforeTimelineStarts(instant);
}
@Override
public Stream<String> getInstants() {
public Stream<HoodieInstant> getInstants() {
return instants.stream();
}
@Override
public Stream<String> getInflightInstants() {
return inflights.stream();
public boolean isBeforeTimelineStarts(String instant) {
Optional<HoodieInstant> firstCommit = firstInstant();
return firstCommit.isPresent() && compareTimestamps(instant,
firstCommit.get().getTimestamp(), LESSER);
}
@Override
public boolean isInstantBeforeTimelineStarts(String instant) {
Optional<String> firstCommit = firstInstant();
return firstCommit.isPresent() && compareInstants(instant, firstCommit.get(), LESSER);
public Optional<byte[]> getInstantDetails(HoodieInstant instant) {
return details.apply(instant);
}
@Override
public void saveInstantAsInflight(String instant) {
log.info("Creating a new in-flight " + getTimelineName() + " " + instant);
// Create the in-flight file
createFileInMetaPath(getInflightFileName(instant), Optional.empty());
}
@Override
public void saveInstantAsComplete(String instant, Optional<byte[]> data) {
log.info("Marking complete " + getTimelineName() + " " + instant);
moveInflightToComplete(instant, data, getCompletedFileName(instant),
HoodieTableMetaClient.makeInflightCommitFileName(instant));
log.info("Completed " + getTimelineName() + " " + instant);
}
@Override
public void revertInstantToInflight(String instant) {
log.info("Reverting instant to inflight " + getTimelineName() + " " + instant);
moveCompleteToInflight(instant, getCompletedFileName(instant),
getInflightFileName(instant));
log.info("Reverted " + getTimelineName() + " " + instant + " to inflight");
}
@Override
public void removeInflightFromTimeline(String instant) {
log.info("Removing in-flight " + getTimelineName() + " " + instant);
String inFlightCommitFileName = getInflightFileName(instant);
Path inFlightCommitFilePath = new Path(metaPath, inFlightCommitFileName);
try {
fs.delete(inFlightCommitFilePath, false);
log.info("Removed in-flight " + getTimelineName() + " " + instant);
} catch (IOException e) {
throw new HoodieIOException(
"Could not remove inflight commit " + inFlightCommitFilePath, e);
}
}
@Override
public Optional<byte[]> readInstantDetails(String instant) {
Path detailPath = new Path(metaPath, getCompletedFileName(instant));
return readDataFromPath(detailPath);
}
/**
* Get the in-flight instant file name
*
* @param instant
* @return
*/
protected abstract String getInflightFileName(String instant);
/**
* Get the completed instant file name
*
* @param instant
* @return
*/
protected abstract String getCompletedFileName(String instant);
/**
* Get the timeline name
*
* @return
*/
protected abstract String getTimelineName();
protected void moveInflightToComplete(String instant, Optional<byte[]> data,
String commitFileName, String inflightFileName) {
Path commitFilePath = new Path(metaPath, commitFileName);
try {
// open a new file and write the commit metadata in
Path inflightCommitFile = new Path(metaPath, inflightFileName);
createFileInMetaPath(inflightFileName, data);
boolean success = fs.rename(inflightCommitFile, commitFilePath);
if (!success) {
throw new HoodieIOException(
"Could not rename " + inflightCommitFile + " to " + commitFilePath);
}
} catch (IOException e) {
throw new HoodieIOException("Could not complete commit " + instant, e);
}
}
protected void moveCompleteToInflight(String instant, String commitFileName,
String inflightFileName) {
Path inFlightCommitFilePath = new Path(metaPath, inflightFileName);
try {
if (!fs.exists(inFlightCommitFilePath)) {
Path commitFilePath = new Path(metaPath, commitFileName);
boolean success = fs.rename(commitFilePath, inFlightCommitFilePath);
if (!success) {
throw new HoodieIOException(
"Could not rename " + commitFilePath + " to " + inFlightCommitFilePath);
}
}
} catch (IOException e) {
throw new HoodieIOException("Could not complete commit revert " + instant, e);
}
}
protected void createFileInMetaPath(String filename, Optional<byte[]> content) {
Path fullPath = new Path(metaPath, filename);
try {
if (!content.isPresent()) {
if (fs.createNewFile(fullPath)) {
log.info("Created a new file in meta path: " + fullPath);
return;
}
} else {
FSDataOutputStream fsout = fs.create(fullPath, true);
fsout.write(content.get());
fsout.close();
return;
}
throw new HoodieIOException("Failed to create file " + fullPath);
} catch (IOException e) {
throw new HoodieIOException("Failed to create file " + fullPath, e);
}
}
protected Optional<byte[]> readDataFromPath(Path detailPath) {
FSDataInputStream is = null;
try {
is = fs.open(detailPath);
return Optional.of(IOUtils.toByteArray(is));
} catch (IOException e) {
throw new HoodieIOException("Could not read commit details from " + detailPath, e);
} finally {
if (is != null) {
Closeables.closeQuietly(is);
}
}
}
@Override
public String toString() {
return this.getClass().getName() + ": " + instants.stream().map(Object::toString)

View File

@@ -0,0 +1,121 @@
/*
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
*
* Licensed 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 com.uber.hoodie.common.table.timeline;
import com.google.common.io.Files;
import com.uber.hoodie.common.table.HoodieTimeline;
import org.apache.hadoop.fs.FileStatus;
import java.io.Serializable;
import java.util.Objects;
import java.util.Optional;
/**
* A Hoodie Instant represents a action done on a hoodie dataset.
* All actions start with a inflight instant and then create a completed instant after done.
*
* @see HoodieTimeline
*/
public class HoodieInstant implements Serializable {
private boolean isInflight = false;
private String action;
private String timestamp;
/**
* Load the instant from the meta FileStatus
* @param fileStatus
*/
public HoodieInstant(FileStatus fileStatus) {
// First read the instant timestamp. [==>20170101193025<==].commit
String fileName = fileStatus.getPath().getName();
String fileExtension = Files.getFileExtension(fileName);
timestamp = fileName.replace("." + fileExtension, "");
// Next read the action for this marker
action = fileExtension;
if(action.equals("inflight")) {
// This is to support backwards compatibility on how in-flight commit files were written
// General rule is inflight extension is .<action>.inflight, but for commit it is .inflight
action = "commit";
isInflight = true;
} else if (action.contains(HoodieTimeline.INFLIGHT_EXTENSION)) {
isInflight = true;
action = action.replace(HoodieTimeline.INFLIGHT_EXTENSION, "");
}
}
public HoodieInstant(boolean isInflight, String action, String timestamp) {
this.isInflight = isInflight;
this.action = action;
this.timestamp = timestamp;
}
public boolean isInflight() {
return isInflight;
}
public String getAction() {
return action;
}
public String getTimestamp() {
return timestamp;
}
/**
* Get the filename for this instant
* @return
*/
public String getFileName() {
if (HoodieTimeline.COMMIT_ACTION.equals(action)) {
return isInflight ?
HoodieTimeline.makeInflightCommitFileName(timestamp) :
HoodieTimeline.makeCommitFileName(timestamp);
} else if (HoodieTimeline.CLEAN_ACTION.equals(action)) {
return isInflight ?
HoodieTimeline.makeInflightCleanerFileName(timestamp) :
HoodieTimeline.makeCleanerFileName(timestamp);
} else if (HoodieTimeline.SAVEPOINT_ACTION.equals(action)) {
return isInflight ?
HoodieTimeline.makeInflightSavePointFileName(timestamp) :
HoodieTimeline.makeSavePointFileName(timestamp);
}
throw new IllegalArgumentException("Cannot get file name for unknown action " + action);
}
@Override
public boolean equals(Object o) {
if (this == o)
return true;
if (o == null || getClass() != o.getClass())
return false;
HoodieInstant that = (HoodieInstant) o;
return isInflight == that.isInflight &&
Objects.equals(action, that.action) &&
Objects.equals(timestamp, that.timestamp);
}
@Override
public int hashCode() {
return Objects.hash(isInflight, action, timestamp);
}
@Override
public String toString() {
return "[" + ((isInflight) ? "==>" : "") + timestamp + "__" + action + "]";
}
}

View File

@@ -1,50 +0,0 @@
/*
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
*
* Licensed 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 com.uber.hoodie.common.table.timeline;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import java.io.IOException;
public class HoodieSavePointTimeline extends HoodieDefaultTimeline {
public HoodieSavePointTimeline(FileSystem fs, String metaPath) {
super(fs, metaPath, HoodieTableMetaClient.SAVEPOINT_EXTENSION);
}
@Override
public HoodieTimeline reload() throws IOException {
return new HoodieSavePointTimeline(fs, metaPath);
}
@Override
protected String getInflightFileName(String instant) {
return HoodieTableMetaClient.makeInflightSavePointFileName(instant);
}
@Override
protected String getCompletedFileName(String instant) {
return HoodieTableMetaClient.makeSavePointFileName(instant);
}
@Override
protected String getTimelineName() {
return "savepoint";
}
}

View File

@@ -20,6 +20,7 @@ import com.uber.hoodie.common.model.HoodieDataFile;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.TableFileSystemView;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.table.timeline.HoodieInstant;
import com.uber.hoodie.exception.HoodieIOException;
import org.apache.commons.lang3.tuple.Pair;
import org.apache.hadoop.fs.FileStatus;
@@ -38,7 +39,7 @@ import java.util.stream.Stream;
/**
* Common abstract implementation for multiple TableFileSystemView Implementations.
* 2 possible implementations are ReadOptimizedView and RealtimeView
*
* <p>
* Concrete implementations extending this abstract class, should only implement
* listDataFilesInPartition which includes files to be included in the view
*
@@ -54,24 +55,26 @@ public abstract class AbstractTableFileSystemView implements TableFileSystemView
public AbstractTableFileSystemView(FileSystem fs, HoodieTableMetaClient metaClient) {
this.metaClient = metaClient;
this.fs = fs;
this.activeCommitTimeline = metaClient.getActiveCommitTimeline();
// Get the active timeline and filter only completed commits
this.activeCommitTimeline =
metaClient.getActiveTimeline().getCommitTimeline().filterCompletedInstants();
}
public Stream<HoodieDataFile> getLatestDataFilesForFileId(final String partitionPath,
String fileId) {
Optional<String> lastInstant = activeCommitTimeline.lastInstant();
Optional<HoodieInstant> lastInstant = activeCommitTimeline.lastInstant();
if (lastInstant.isPresent()) {
return streamLatestVersionInPartition(partitionPath, lastInstant.get())
return getLatestVersionInPartition(partitionPath, lastInstant.get().getTimestamp())
.filter(hoodieDataFile -> hoodieDataFile.getFileId().equals(fileId));
}
return Stream.empty();
}
@Override
public Stream<HoodieDataFile> streamLatestVersionInPartition(String partitionPathStr,
public Stream<HoodieDataFile> getLatestVersionInPartition(String partitionPathStr,
String maxCommitTime) {
try {
return streamLatestVersionsBeforeOrOn(listDataFilesInPartition(partitionPathStr),
return getLatestVersionsBeforeOrOn(listDataFilesInPartition(partitionPathStr),
maxCommitTime);
} catch (IOException e) {
throw new HoodieIOException(
@@ -81,11 +84,11 @@ public abstract class AbstractTableFileSystemView implements TableFileSystemView
@Override
public Stream<List<HoodieDataFile>> streamEveryVersionInPartition(String partitionPath) {
public Stream<List<HoodieDataFile>> getEveryVersionInPartition(String partitionPath) {
try {
if(activeCommitTimeline.lastInstant().isPresent()) {
return streamFilesByFileId(listDataFilesInPartition(partitionPath),
activeCommitTimeline.lastInstant().get());
if (activeCommitTimeline.lastInstant().isPresent()) {
return getFilesByFileId(listDataFilesInPartition(partitionPath),
activeCommitTimeline.lastInstant().get().getTimestamp());
}
return Stream.empty();
} catch (IOException e) {
@@ -98,13 +101,14 @@ public abstract class AbstractTableFileSystemView implements TableFileSystemView
throws IOException;
@Override
public Stream<HoodieDataFile> streamLatestVersionInRange(FileStatus[] fileStatuses,
public Stream<HoodieDataFile> getLatestVersionInRange(FileStatus[] fileStatuses,
List<String> commitsToReturn) {
if (!activeCommitTimeline.hasInstants() || commitsToReturn.isEmpty()) {
if (activeCommitTimeline.empty() || commitsToReturn.isEmpty()) {
return Stream.empty();
}
try {
return streamFilesByFileId(fileStatuses, activeCommitTimeline.lastInstant().get())
return getFilesByFileId(fileStatuses,
activeCommitTimeline.lastInstant().get().getTimestamp())
.map((Function<List<HoodieDataFile>, Optional<HoodieDataFile>>) fss -> {
for (HoodieDataFile fs : fss) {
if (commitsToReturn.contains(fs.getCommitTime())) {
@@ -120,17 +124,18 @@ public abstract class AbstractTableFileSystemView implements TableFileSystemView
}
@Override
public Stream<HoodieDataFile> streamLatestVersionsBeforeOrOn(FileStatus[] fileStatuses,
public Stream<HoodieDataFile> getLatestVersionsBeforeOrOn(FileStatus[] fileStatuses,
String maxCommitToReturn) {
try {
if (!activeCommitTimeline.hasInstants()) {
if (activeCommitTimeline.empty()) {
return Stream.empty();
}
return streamFilesByFileId(fileStatuses, activeCommitTimeline.lastInstant().get())
return getFilesByFileId(fileStatuses,
activeCommitTimeline.lastInstant().get().getTimestamp())
.map((Function<List<HoodieDataFile>, Optional<HoodieDataFile>>) fss -> {
for (HoodieDataFile fs1 : fss) {
if (activeCommitTimeline
.compareInstants(fs1.getCommitTime(), maxCommitToReturn,
.compareTimestamps(fs1.getCommitTime(), maxCommitToReturn,
HoodieTimeline.LESSER_OR_EQUAL)) {
return Optional.of(fs1);
}
@@ -143,19 +148,20 @@ public abstract class AbstractTableFileSystemView implements TableFileSystemView
}
@Override
public Stream<HoodieDataFile> streamLatestVersions(FileStatus[] fileStatuses) {
public Stream<HoodieDataFile> getLatestVersions(FileStatus[] fileStatuses) {
try {
if (!activeCommitTimeline.hasInstants()) {
if (activeCommitTimeline.empty()) {
return Stream.empty();
}
return streamFilesByFileId(fileStatuses, activeCommitTimeline.lastInstant().get())
return getFilesByFileId(fileStatuses,
activeCommitTimeline.lastInstant().get().getTimestamp())
.map(statuses -> statuses.get(0));
} catch (IOException e) {
throw new HoodieIOException("Could not filter files for latest version ", e);
}
}
protected Stream<List<HoodieDataFile>> streamFilesByFileId(FileStatus[] files,
protected Stream<List<HoodieDataFile>> getFilesByFileId(FileStatus[] files,
String maxCommitTime) throws IOException {
return groupFilesByFileId(files, maxCommitTime).values().stream();
}
@@ -173,7 +179,7 @@ public abstract class AbstractTableFileSystemView implements TableFileSystemView
return Arrays.stream(files).flatMap(fileStatus -> {
HoodieDataFile dataFile = new HoodieDataFile(fileStatus);
if (activeCommitTimeline.containsOrBeforeTimelineStarts(dataFile.getCommitTime())
&& activeCommitTimeline.compareInstants(dataFile.getCommitTime(), maxCommitTime,
&& activeCommitTimeline.compareTimestamps(dataFile.getCommitTime(), maxCommitTime,
HoodieTimeline.LESSER_OR_EQUAL)) {
return Stream.of(Pair.of(dataFile.getFileId(), dataFile));
}

View File

@@ -36,7 +36,7 @@ public class ReadOptimizedTableView extends AbstractTableFileSystemView {
Path partitionPath = new Path(metaClient.getBasePath(), partitionPathStr);
try {
return fs.listStatus(partitionPath, path -> path.getName()
.contains(metaClient.getTableConfig().getROStorageFormat().getFileExtension()));
.contains(metaClient.getTableConfig().getROFileFormat().getFileExtension()));
} catch (IOException e) {
throw new HoodieIOException(
"Failed to list data files in partition " + partitionPathStr, e);

View File

@@ -16,6 +16,10 @@
package com.uber.hoodie.common.util;
import com.google.common.base.Preconditions;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.table.timeline.HoodieInstant;
import com.uber.hoodie.exception.HoodieIOException;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
@@ -24,6 +28,7 @@ import org.apache.hadoop.fs.Path;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import java.io.File;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
@@ -91,18 +96,27 @@ public class FSUtils {
/**
* Obtain all the partition paths, that are present in this table.
*/
public static List<String> getAllPartitionPaths(FileSystem fs, String basePath) throws IOException {
public static List<String> getAllPartitionPaths(FileSystem fs, String basePath)
throws IOException {
List<String> partitionsToClean = new ArrayList<>();
// TODO(vc): For now, assume partitions are two levels down from base path.
FileStatus[] folders = fs.globStatus(new Path(basePath + "/*/*/*"));
for (FileStatus status : folders) {
Path path = status.getPath();
partitionsToClean.add(String.format("%s/%s/%s",
path.getParent().getParent().getName(),
path.getParent().getName(),
path.getName()));
partitionsToClean.add(String.format("%s/%s/%s", path.getParent().getParent().getName(),
path.getParent().getName(), path.getName()));
}
return partitionsToClean;
}
public static String getFileExtension(String fullName) {
Preconditions.checkNotNull(fullName);
String fileName = (new File(fullName)).getName();
int dotIndex = fileName.indexOf('.');
return dotIndex == -1 ? "" : fileName.substring(dotIndex);
}
public static String getInstantTime(String name) {
return name.replace(getFileExtension(name), "");
}
}