1
0

HUDI-138 - Meta Files handling also need to support consistency guard

This commit is contained in:
Balaji Varadarajan
2019-06-20 18:05:01 -07:00
committed by Balaji Varadarajan
parent 621c246fa9
commit 5823c1ebd7
21 changed files with 482 additions and 167 deletions

View File

@@ -19,6 +19,7 @@
package com.uber.hoodie.cli; package com.uber.hoodie.cli;
import com.uber.hoodie.common.table.HoodieTableMetaClient; import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.util.ConsistencyGuardConfig;
import com.uber.hoodie.common.util.FSUtils; import com.uber.hoodie.common.util.FSUtils;
import java.io.IOException; import java.io.IOException;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
@@ -27,8 +28,10 @@ import org.apache.hadoop.fs.FileSystem;
public class HoodieCLI { public class HoodieCLI {
public static Configuration conf; public static Configuration conf;
public static ConsistencyGuardConfig consistencyGuardConfig = ConsistencyGuardConfig.newBuilder().build();
public static FileSystem fs; public static FileSystem fs;
public static CLIState state = CLIState.INIT; public static CLIState state = CLIState.INIT;
public static String basePath;
public static HoodieTableMetaClient tableMetadata; public static HoodieTableMetaClient tableMetadata;
public static HoodieTableMetaClient syncTableMetadata; public static HoodieTableMetaClient syncTableMetadata;
@@ -37,6 +40,18 @@ public class HoodieCLI {
INIT, DATASET, SYNC INIT, DATASET, SYNC
} }
public static void setConsistencyGuardConfig(ConsistencyGuardConfig config) {
consistencyGuardConfig = config;
}
private static void setTableMetaClient(HoodieTableMetaClient tableMetadata) {
HoodieCLI.tableMetadata = tableMetadata;
}
private static void setBasePath(String basePath) {
HoodieCLI.basePath = basePath;
}
public static boolean initConf() { public static boolean initConf() {
if (HoodieCLI.conf == null) { if (HoodieCLI.conf == null) {
HoodieCLI.conf = FSUtils.prepareHadoopConf(new Configuration()); HoodieCLI.conf = FSUtils.prepareHadoopConf(new Configuration());
@@ -47,11 +62,16 @@ public class HoodieCLI {
public static void initFS(boolean force) throws IOException { public static void initFS(boolean force) throws IOException {
if (fs == null || force) { if (fs == null || force) {
fs = FileSystem.get(conf); fs = (tableMetadata != null) ? tableMetadata.getFs() : FileSystem.get(conf);
} }
} }
public static void setTableMetadata(HoodieTableMetaClient tableMetadata) { public static void refreshTableMetadata() {
HoodieCLI.tableMetadata = tableMetadata; setTableMetaClient(new HoodieTableMetaClient(HoodieCLI.conf, basePath, false, HoodieCLI.consistencyGuardConfig));
}
public static void connectTo(String basePath) {
setBasePath(basePath);
refreshTableMetadata();
} }
} }

View File

@@ -23,7 +23,6 @@ import com.uber.hoodie.avro.model.HoodieCleanPartitionMetadata;
import com.uber.hoodie.cli.HoodieCLI; import com.uber.hoodie.cli.HoodieCLI;
import com.uber.hoodie.cli.HoodiePrintHelper; import com.uber.hoodie.cli.HoodiePrintHelper;
import com.uber.hoodie.cli.TableHeader; import com.uber.hoodie.cli.TableHeader;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline; import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline; import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
import com.uber.hoodie.common.table.timeline.HoodieInstant; import com.uber.hoodie.common.table.timeline.HoodieInstant;
@@ -91,9 +90,8 @@ public class CleansCommand implements CommandMarker {
@CliCommand(value = "cleans refresh", help = "Refresh the commits") @CliCommand(value = "cleans refresh", help = "Refresh the commits")
public String refreshCleans() throws IOException { public String refreshCleans() throws IOException {
HoodieTableMetaClient metadata = new HoodieTableMetaClient(HoodieCLI.conf, HoodieCLI.tableMetadata.getBasePath()); HoodieCLI.refreshTableMetadata();
HoodieCLI.setTableMetadata(metadata); return "Metadata for table " + HoodieCLI.tableMetadata.getTableConfig().getTableName() + " refreshed.";
return "Metadata for table " + metadata.getTableConfig().getTableName() + " refreshed.";
} }
@CliCommand(value = "clean showpartitions", help = "Show partition level details of a clean") @CliCommand(value = "clean showpartitions", help = "Show partition level details of a clean")

View File

@@ -115,9 +115,8 @@ public class CommitsCommand implements CommandMarker {
@CliCommand(value = "commits refresh", help = "Refresh the commits") @CliCommand(value = "commits refresh", help = "Refresh the commits")
public String refreshCommits() throws IOException { public String refreshCommits() throws IOException {
HoodieTableMetaClient metadata = new HoodieTableMetaClient(HoodieCLI.conf, HoodieCLI.tableMetadata.getBasePath()); HoodieCLI.refreshTableMetadata();
HoodieCLI.setTableMetadata(metadata); return "Metadata for table " + HoodieCLI.tableMetadata.getTableConfig().getTableName() + " refreshed.";
return "Metadata for table " + metadata.getTableConfig().getTableName() + " refreshed.";
} }
@CliCommand(value = "commit rollback", help = "Rollback a commit") @CliCommand(value = "commit rollback", help = "Rollback a commit")

View File

@@ -23,6 +23,7 @@ import com.uber.hoodie.cli.HoodiePrintHelper;
import com.uber.hoodie.cli.TableHeader; import com.uber.hoodie.cli.TableHeader;
import com.uber.hoodie.common.model.HoodieTableType; import com.uber.hoodie.common.model.HoodieTableType;
import com.uber.hoodie.common.table.HoodieTableMetaClient; import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.util.ConsistencyGuardConfig;
import com.uber.hoodie.exception.DatasetNotFoundException; import com.uber.hoodie.exception.DatasetNotFoundException;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
@@ -39,11 +40,25 @@ public class DatasetsCommand implements CommandMarker {
@CliCommand(value = "connect", help = "Connect to a hoodie dataset") @CliCommand(value = "connect", help = "Connect to a hoodie dataset")
public String connect( public String connect(
@CliOption(key = {"path"}, mandatory = true, help = "Base Path of the dataset") final String path) @CliOption(key = {"path"}, mandatory = true, help = "Base Path of the dataset") final String path,
throws IOException { @CliOption(key = {"eventuallyConsistent"}, mandatory = false, unspecifiedDefaultValue = "false",
boolean initialized = HoodieCLI.initConf(); help = "Enable eventual consistency") final boolean eventuallyConsistent,
HoodieCLI.initFS(initialized); @CliOption(key = {"initialCheckIntervalMs"}, mandatory = false, unspecifiedDefaultValue = "2000",
HoodieCLI.setTableMetadata(new HoodieTableMetaClient(HoodieCLI.conf, path)); help = "Initial wait time for eventual consistency") final Integer initialConsistencyIntervalMs,
@CliOption(key = {"maxCheckIntervalMs"}, mandatory = false, unspecifiedDefaultValue = "300000",
help = "Max wait time for eventual consistency") final Integer maxConsistencyIntervalMs,
@CliOption(key = {"maxCheckIntervalMs"}, mandatory = false, unspecifiedDefaultValue = "7",
help = "Max checks for eventual consistency") final Integer maxConsistencyChecks) throws IOException {
HoodieCLI.setConsistencyGuardConfig(
ConsistencyGuardConfig.newBuilder()
.withConsistencyCheckEnabled(eventuallyConsistent)
.withInitialConsistencyCheckIntervalMs(initialConsistencyIntervalMs)
.withMaxConsistencyCheckIntervalMs(maxConsistencyIntervalMs)
.withMaxConsistencyChecks(maxConsistencyChecks)
.build());
HoodieCLI.initConf();
HoodieCLI.connectTo(path);
HoodieCLI.initFS(true);
HoodieCLI.state = HoodieCLI.CLIState.DATASET; HoodieCLI.state = HoodieCLI.CLIState.DATASET;
return "Metadata for table " + HoodieCLI.tableMetadata.getTableConfig().getTableName() + " loaded"; return "Metadata for table " + HoodieCLI.tableMetadata.getTableConfig().getTableName() + " loaded";
} }
@@ -85,7 +100,7 @@ public class DatasetsCommand implements CommandMarker {
HoodieTableMetaClient.initTableType(HoodieCLI.conf, path, tableType, name, payloadClass); HoodieTableMetaClient.initTableType(HoodieCLI.conf, path, tableType, name, payloadClass);
// Now connect to ensure loading works // Now connect to ensure loading works
return connect(path); return connect(path, false, 0, 0, 0);
} }
@CliAvailabilityIndicator({"desc"}) @CliAvailabilityIndicator({"desc"})

View File

@@ -23,7 +23,6 @@ import com.uber.hoodie.cli.HoodieCLI;
import com.uber.hoodie.cli.HoodiePrintHelper; import com.uber.hoodie.cli.HoodiePrintHelper;
import com.uber.hoodie.cli.utils.InputStreamConsumer; import com.uber.hoodie.cli.utils.InputStreamConsumer;
import com.uber.hoodie.cli.utils.SparkUtil; import com.uber.hoodie.cli.utils.SparkUtil;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline; import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline; import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
import com.uber.hoodie.common.table.timeline.HoodieInstant; import com.uber.hoodie.common.table.timeline.HoodieInstant;
@@ -133,9 +132,8 @@ public class SavepointsCommand implements CommandMarker {
@CliCommand(value = "savepoints refresh", help = "Refresh the savepoints") @CliCommand(value = "savepoints refresh", help = "Refresh the savepoints")
public String refreshMetaClient() throws IOException { public String refreshMetaClient() throws IOException {
HoodieTableMetaClient metadata = new HoodieTableMetaClient(HoodieCLI.conf, HoodieCLI.tableMetadata.getBasePath()); HoodieCLI.refreshTableMetadata();
HoodieCLI.setTableMetadata(metadata); return "Metadata for table " + HoodieCLI.tableMetadata.getTableConfig().getTableName() + " refreshed.";
return "Metadata for table " + metadata.getTableConfig().getTableName() + " refreshed.";
} }
private static HoodieWriteClient createHoodieClient(JavaSparkContext jsc, String basePath) throws Exception { private static HoodieWriteClient createHoodieClient(JavaSparkContext jsc, String basePath) throws Exception {

View File

@@ -19,6 +19,8 @@
package com.uber.hoodie; package com.uber.hoodie;
import com.uber.hoodie.client.embedded.EmbeddedTimelineService; import com.uber.hoodie.client.embedded.EmbeddedTimelineService;
import com.uber.hoodie.client.utils.ClientUtils;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.util.FSUtils; import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.config.HoodieWriteConfig;
import java.io.IOException; import java.io.IOException;
@@ -117,4 +119,8 @@ public abstract class AbstractHoodieClient implements Serializable {
public Optional<EmbeddedTimelineService> getTimelineServer() { public Optional<EmbeddedTimelineService> getTimelineServer() {
return timelineServer; return timelineServer;
} }
protected HoodieTableMetaClient createMetaClient(boolean loadActiveTimelineOnLoad) {
return ClientUtils.createMetaClient(jsc, config, loadActiveTimelineOnLoad);
}
} }

View File

@@ -114,7 +114,7 @@ public class CompactionAdminClient extends AbstractHoodieClient {
*/ */
public List<RenameOpResult> unscheduleCompactionPlan( public List<RenameOpResult> unscheduleCompactionPlan(
String compactionInstant, boolean skipValidation, int parallelism, boolean dryRun) throws Exception { String compactionInstant, boolean skipValidation, int parallelism, boolean dryRun) throws Exception {
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); HoodieTableMetaClient metaClient = createMetaClient(false);
List<Pair<HoodieLogFile, HoodieLogFile>> renameActions = List<Pair<HoodieLogFile, HoodieLogFile>> renameActions =
getRenamingActionsForUnschedulingCompactionPlan(metaClient, compactionInstant, parallelism, getRenamingActionsForUnschedulingCompactionPlan(metaClient, compactionInstant, parallelism,
Optional.absent(), skipValidation); Optional.absent(), skipValidation);
@@ -156,7 +156,7 @@ public class CompactionAdminClient extends AbstractHoodieClient {
*/ */
public List<RenameOpResult> unscheduleCompactionFileId(HoodieFileGroupId fgId, public List<RenameOpResult> unscheduleCompactionFileId(HoodieFileGroupId fgId,
boolean skipValidation, boolean dryRun) throws Exception { boolean skipValidation, boolean dryRun) throws Exception {
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); HoodieTableMetaClient metaClient = createMetaClient(false);
List<Pair<HoodieLogFile, HoodieLogFile>> renameActions = List<Pair<HoodieLogFile, HoodieLogFile>> renameActions =
getRenamingActionsForUnschedulingCompactionForFileId(metaClient, fgId, getRenamingActionsForUnschedulingCompactionForFileId(metaClient, fgId,
Optional.absent(), skipValidation); Optional.absent(), skipValidation);
@@ -198,7 +198,7 @@ public class CompactionAdminClient extends AbstractHoodieClient {
*/ */
public List<RenameOpResult> repairCompaction(String compactionInstant, public List<RenameOpResult> repairCompaction(String compactionInstant,
int parallelism, boolean dryRun) throws Exception { int parallelism, boolean dryRun) throws Exception {
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); HoodieTableMetaClient metaClient = createMetaClient(false);
List<ValidationOpResult> validationResults = List<ValidationOpResult> validationResults =
validateCompactionPlan(metaClient, compactionInstant, parallelism); validateCompactionPlan(metaClient, compactionInstant, parallelism);
List<ValidationOpResult> failed = validationResults.stream() List<ValidationOpResult> failed = validationResults.stream()

View File

@@ -76,7 +76,6 @@ import java.util.Map;
import java.util.Optional; import java.util.Optional;
import java.util.stream.Collectors; import java.util.stream.Collectors;
import java.util.stream.IntStream; import java.util.stream.IntStream;
import org.apache.hadoop.conf.Configuration;
import org.apache.log4j.LogManager; import org.apache.log4j.LogManager;
import org.apache.log4j.Logger; import org.apache.log4j.Logger;
import org.apache.spark.Partitioner; import org.apache.spark.Partitioner;
@@ -153,7 +152,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
public JavaRDD<HoodieRecord<T>> filterExists(JavaRDD<HoodieRecord<T>> hoodieRecords) { public JavaRDD<HoodieRecord<T>> filterExists(JavaRDD<HoodieRecord<T>> hoodieRecords) {
// Create a Hoodie table which encapsulated the commits and files visible // Create a Hoodie table which encapsulated the commits and files visible
HoodieTable<T> table = HoodieTable.getHoodieTable( HoodieTable<T> table = HoodieTable.getHoodieTable(
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config, jsc); createMetaClient(true), config, jsc);
JavaRDD<HoodieRecord<T>> recordsWithLocation = index.tagLocation(hoodieRecords, jsc, table); JavaRDD<HoodieRecord<T>> recordsWithLocation = index.tagLocation(hoodieRecords, jsc, table);
return recordsWithLocation.filter(v1 -> !v1.isCurrentLocationKnown()); return recordsWithLocation.filter(v1 -> !v1.isCurrentLocationKnown());
@@ -471,9 +470,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
JavaRDD<WriteStatus> statuses = index.updateLocation(writeStatusRDD, jsc, table); JavaRDD<WriteStatus> statuses = index.updateLocation(writeStatusRDD, jsc, table);
// Trigger the insert and collect statuses // Trigger the insert and collect statuses
statuses = statuses.persist(config.getWriteStatusStorageLevel()); statuses = statuses.persist(config.getWriteStatusStorageLevel());
commitOnAutoCommit(commitTime, statuses, commitOnAutoCommit(commitTime, statuses, table.getMetaClient().getCommitActionType());
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true)
.getCommitActionType());
return statuses; return statuses;
} }
@@ -496,7 +493,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
*/ */
public boolean commit(String commitTime, JavaRDD<WriteStatus> writeStatuses, public boolean commit(String commitTime, JavaRDD<WriteStatus> writeStatuses,
Optional<Map<String, String>> extraMetadata) { Optional<Map<String, String>> extraMetadata) {
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true); HoodieTableMetaClient metaClient = createMetaClient(false);
return commit(commitTime, writeStatuses, extraMetadata, metaClient.getCommitActionType()); return commit(commitTime, writeStatuses, extraMetadata, metaClient.getCommitActionType());
} }
@@ -506,7 +503,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
logger.info("Commiting " + commitTime); logger.info("Commiting " + commitTime);
// Create a Hoodie table which encapsulated the commits and files visible // Create a Hoodie table which encapsulated the commits and files visible
HoodieTable<T> table = HoodieTable.getHoodieTable( HoodieTable<T> table = HoodieTable.getHoodieTable(
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config, jsc); createMetaClient(true), config, jsc);
HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); HoodieActiveTimeline activeTimeline = table.getActiveTimeline();
HoodieCommitMetadata metadata = new HoodieCommitMetadata(); HoodieCommitMetadata metadata = new HoodieCommitMetadata();
@@ -536,7 +533,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
// We cannot have unbounded commit files. Archive commits if we have to archive // We cannot have unbounded commit files. Archive commits if we have to archive
HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(config, HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(config,
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true)); createMetaClient(true));
archiveLog.archiveIfRequired(jsc); archiveLog.archiveIfRequired(jsc);
if (config.isAutoClean()) { if (config.isAutoClean()) {
// Call clean to cleanup if there is anything to cleanup after the commit, // Call clean to cleanup if there is anything to cleanup after the commit,
@@ -580,7 +577,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
*/ */
public boolean savepoint(String user, String comment) { public boolean savepoint(String user, String comment) {
HoodieTable<T> table = HoodieTable.getHoodieTable( HoodieTable<T> table = HoodieTable.getHoodieTable(
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config, jsc); createMetaClient(true), config, jsc);
if (table.getCompletedCommitsTimeline().empty()) { if (table.getCompletedCommitsTimeline().empty()) {
throw new HoodieSavepointException("Could not savepoint. Commit timeline is empty"); throw new HoodieSavepointException("Could not savepoint. Commit timeline is empty");
} }
@@ -610,7 +607,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
*/ */
public boolean savepoint(String commitTime, String user, String comment) { public boolean savepoint(String commitTime, String user, String comment) {
HoodieTable<T> table = HoodieTable.getHoodieTable( HoodieTable<T> table = HoodieTable.getHoodieTable(
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config, jsc); createMetaClient(true), config, jsc);
if (table.getMetaClient().getTableType() == HoodieTableType.MERGE_ON_READ) { if (table.getMetaClient().getTableType() == HoodieTableType.MERGE_ON_READ) {
throw new UnsupportedOperationException("Savepointing is not supported or MergeOnRead table types"); throw new UnsupportedOperationException("Savepointing is not supported or MergeOnRead table types");
} }
@@ -674,7 +671,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
*/ */
public void deleteSavepoint(String savepointTime) { public void deleteSavepoint(String savepointTime) {
HoodieTable<T> table = HoodieTable.getHoodieTable( HoodieTable<T> table = HoodieTable.getHoodieTable(
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config, jsc); createMetaClient(true), config, jsc);
if (table.getMetaClient().getTableType() == HoodieTableType.MERGE_ON_READ) { if (table.getMetaClient().getTableType() == HoodieTableType.MERGE_ON_READ) {
throw new UnsupportedOperationException("Savepointing is not supported or MergeOnRead table types"); throw new UnsupportedOperationException("Savepointing is not supported or MergeOnRead table types");
} }
@@ -705,7 +702,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
*/ */
private void deleteRequestedCompaction(String compactionTime) { private void deleteRequestedCompaction(String compactionTime) {
HoodieTable<T> table = HoodieTable.getHoodieTable( HoodieTable<T> table = HoodieTable.getHoodieTable(
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config, jsc); createMetaClient(true), config, jsc);
HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); HoodieActiveTimeline activeTimeline = table.getActiveTimeline();
HoodieInstant compactionRequestedInstant = HoodieInstant compactionRequestedInstant =
new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, compactionTime); new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, compactionTime);
@@ -734,7 +731,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
*/ */
public boolean rollbackToSavepoint(String savepointTime) { public boolean rollbackToSavepoint(String savepointTime) {
HoodieTable<T> table = HoodieTable.getHoodieTable( HoodieTable<T> table = HoodieTable.getHoodieTable(
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config, jsc); createMetaClient(true), config, jsc);
HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); HoodieActiveTimeline activeTimeline = table.getActiveTimeline();
// Rollback to savepoint is expected to be a manual operation and no concurrent write or compaction is expected // Rollback to savepoint is expected to be a manual operation and no concurrent write or compaction is expected
@@ -788,7 +785,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
// Create a Hoodie table which encapsulated the commits and files visible // Create a Hoodie table which encapsulated the commits and files visible
HoodieTable<T> table = HoodieTable.getHoodieTable( HoodieTable<T> table = HoodieTable.getHoodieTable(
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config, jsc); createMetaClient(true), config, jsc);
// Get all the commits on the timeline after the provided commit time // Get all the commits on the timeline after the provided commit time
List<HoodieInstant> instantsToRollback = table.getActiveTimeline().getCommitsAndCompactionTimeline().getInstants() List<HoodieInstant> instantsToRollback = table.getActiveTimeline().getCommitsAndCompactionTimeline().getInstants()
.filter(instant -> HoodieActiveTimeline.GREATER.test(instant.getTimestamp(), instantTime)) .filter(instant -> HoodieActiveTimeline.GREATER.test(instant.getTimestamp(), instantTime))
@@ -848,7 +845,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
private List<HoodieRollbackStat> doRollbackAndGetStats(final String commitToRollback) throws private List<HoodieRollbackStat> doRollbackAndGetStats(final String commitToRollback) throws
IOException { IOException {
HoodieTable<T> table = HoodieTable.getHoodieTable( HoodieTable<T> table = HoodieTable.getHoodieTable(
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config, jsc); createMetaClient(true), config, jsc);
HoodieTimeline inflightCommitTimeline = table.getInflightCommitTimeline(); HoodieTimeline inflightCommitTimeline = table.getInflightCommitTimeline();
HoodieTimeline commitTimeline = table.getCompletedCommitsTimeline(); HoodieTimeline commitTimeline = table.getCompletedCommitsTimeline();
// Check if any of the commits is a savepoint - do not allow rollback on those commits // Check if any of the commits is a savepoint - do not allow rollback on those commits
@@ -899,7 +896,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
private void finishRollback(final Timer.Context context, List<HoodieRollbackStat> rollbackStats, private void finishRollback(final Timer.Context context, List<HoodieRollbackStat> rollbackStats,
List<String> commitsToRollback, final String startRollbackTime) throws IOException { List<String> commitsToRollback, final String startRollbackTime) throws IOException {
HoodieTable<T> table = HoodieTable.getHoodieTable( HoodieTable<T> table = HoodieTable.getHoodieTable(
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config, jsc); createMetaClient(true), config, jsc);
Optional<Long> durationInMs = Optional.empty(); Optional<Long> durationInMs = Optional.empty();
Long numFilesDeleted = rollbackStats.stream().mapToLong(stat -> stat.getSuccessDeleteFiles().size()).sum(); Long numFilesDeleted = rollbackStats.stream().mapToLong(stat -> stat.getSuccessDeleteFiles().size()).sum();
if (context != null) { if (context != null) {
@@ -925,7 +922,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
private void finishRestore(final Timer.Context context, Map<String, List<HoodieRollbackStat>> commitToStats, private void finishRestore(final Timer.Context context, Map<String, List<HoodieRollbackStat>> commitToStats,
List<String> commitsToRollback, final String startRestoreTime, final String restoreToInstant) throws IOException { List<String> commitsToRollback, final String startRestoreTime, final String restoreToInstant) throws IOException {
HoodieTable<T> table = HoodieTable.getHoodieTable( HoodieTable<T> table = HoodieTable.getHoodieTable(
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config, jsc); createMetaClient(true), config, jsc);
Optional<Long> durationInMs = Optional.empty(); Optional<Long> durationInMs = Optional.empty();
Long numFilesDeleted = 0L; Long numFilesDeleted = 0L;
for (Map.Entry<String, List<HoodieRollbackStat>> commitToStat : commitToStats.entrySet()) { for (Map.Entry<String, List<HoodieRollbackStat>> commitToStat : commitToStats.entrySet()) {
@@ -1001,7 +998,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
// Create a Hoodie table which encapsulated the commits and files visible // Create a Hoodie table which encapsulated the commits and files visible
HoodieTable<T> table = HoodieTable.getHoodieTable( HoodieTable<T> table = HoodieTable.getHoodieTable(
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config, jsc); createMetaClient(true), config, jsc);
List<HoodieCleanStat> cleanStats = table.clean(jsc); List<HoodieCleanStat> cleanStats = table.clean(jsc);
if (cleanStats.isEmpty()) { if (cleanStats.isEmpty()) {
@@ -1053,7 +1050,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
rollbackInflightCommits(); rollbackInflightCommits();
} }
logger.info("Generate a new instant time " + instantTime); logger.info("Generate a new instant time " + instantTime);
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath()); HoodieTableMetaClient metaClient = createMetaClient(true);
// if there are pending compactions, their instantTime must not be greater than that of this instant time // if there are pending compactions, their instantTime must not be greater than that of this instant time
metaClient.getActiveTimeline().filterPendingCompactionTimeline().lastInstant().ifPresent(latestPending -> { metaClient.getActiveTimeline().filterPendingCompactionTimeline().lastInstant().ifPresent(latestPending -> {
Preconditions.checkArgument( Preconditions.checkArgument(
@@ -1086,8 +1083,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
*/ */
public boolean scheduleCompactionAtInstant(String instantTime, Optional<Map<String, String>> extraMetadata) public boolean scheduleCompactionAtInstant(String instantTime, Optional<Map<String, String>> extraMetadata)
throws IOException { throws IOException {
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), HoodieTableMetaClient metaClient = createMetaClient(true);
config.getBasePath(), true);
// if there are inflight writes, their instantTime must not be less than that of compaction instant time // if there are inflight writes, their instantTime must not be less than that of compaction instant time
metaClient.getCommitsTimeline().filterInflightsExcludingCompaction().firstInstant().ifPresent(earliestInflight -> { metaClient.getCommitsTimeline().filterInflightsExcludingCompaction().firstInstant().ifPresent(earliestInflight -> {
Preconditions.checkArgument( Preconditions.checkArgument(
@@ -1130,8 +1126,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
*/ */
public void commitCompaction(String compactionInstantTime, JavaRDD<WriteStatus> writeStatuses, public void commitCompaction(String compactionInstantTime, JavaRDD<WriteStatus> writeStatuses,
Optional<Map<String, String>> extraMetadata) throws IOException { Optional<Map<String, String>> extraMetadata) throws IOException {
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), HoodieTableMetaClient metaClient = createMetaClient(true);
config.getBasePath(), true);
HoodieTable<T> table = HoodieTable.getHoodieTable(metaClient, config, jsc); HoodieTable<T> table = HoodieTable.getHoodieTable(metaClient, config, jsc);
HoodieActiveTimeline timeline = metaClient.getActiveTimeline(); HoodieActiveTimeline timeline = metaClient.getActiveTimeline();
HoodieCompactionPlan compactionPlan = AvroUtils.deserializeCompactionPlan( HoodieCompactionPlan compactionPlan = AvroUtils.deserializeCompactionPlan(
@@ -1178,7 +1173,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
*/ */
private void rollbackInflightCommits() { private void rollbackInflightCommits() {
HoodieTable<T> table = HoodieTable.getHoodieTable( HoodieTable<T> table = HoodieTable.getHoodieTable(
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config, jsc); createMetaClient(true), config, jsc);
HoodieTimeline inflightTimeline = table.getMetaClient().getCommitsTimeline().filterInflightsExcludingCompaction(); HoodieTimeline inflightTimeline = table.getMetaClient().getCommitsTimeline().filterInflightsExcludingCompaction();
List<String> commits = inflightTimeline.getInstants().map(HoodieInstant::getTimestamp) List<String> commits = inflightTimeline.getInstants().map(HoodieInstant::getTimestamp)
.collect(Collectors.toList()); .collect(Collectors.toList());
@@ -1190,11 +1185,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
private HoodieTable getTableAndInitCtx(JavaRDD<HoodieRecord<T>> records) { private HoodieTable getTableAndInitCtx(JavaRDD<HoodieRecord<T>> records) {
// Create a Hoodie table which encapsulated the commits and files visible // Create a Hoodie table which encapsulated the commits and files visible
HoodieTable table = HoodieTable.getHoodieTable( HoodieTable table = HoodieTable.getHoodieTable(createMetaClient(true), config, jsc);
new HoodieTableMetaClient(
// Clone Configuration here. Otherwise we could see ConcurrentModificationException (race) in multi-threaded
// execution (HoodieDeltaStreamer) when Configuration gets serialized by Spark.
new Configuration(jsc.hadoopConfiguration()), config.getBasePath(), true), config, jsc);
if (table.getMetaClient().getCommitActionType().equals(HoodieTimeline.COMMIT_ACTION)) { if (table.getMetaClient().getCommitActionType().equals(HoodieTimeline.COMMIT_ACTION)) {
writeContext = metrics.getCommitCtx(); writeContext = metrics.getCommitCtx();
} else { } else {
@@ -1214,8 +1205,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
*/ */
private JavaRDD<WriteStatus> compact(String compactionInstantTime, boolean autoCommit) throws IOException { private JavaRDD<WriteStatus> compact(String compactionInstantTime, boolean autoCommit) throws IOException {
// Create a Hoodie table which encapsulated the commits and files visible // Create a Hoodie table which encapsulated the commits and files visible
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), HoodieTableMetaClient metaClient = createMetaClient(true);
config.getBasePath(), true);
HoodieTable<T> table = HoodieTable.getHoodieTable(metaClient, config, jsc); HoodieTable<T> table = HoodieTable.getHoodieTable(metaClient, config, jsc);
HoodieTimeline pendingCompactionTimeline = metaClient.getActiveTimeline().filterPendingCompactionTimeline(); HoodieTimeline pendingCompactionTimeline = metaClient.getActiveTimeline().filterPendingCompactionTimeline();
HoodieInstant inflightInstant = HoodieTimeline.getCompactionInflightInstant(compactionInstantTime); HoodieInstant inflightInstant = HoodieTimeline.getCompactionInflightInstant(compactionInstantTime);
@@ -1223,7 +1213,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
//inflight compaction - Needs to rollback first deleting new parquet files before we run compaction. //inflight compaction - Needs to rollback first deleting new parquet files before we run compaction.
rollbackInflightCompaction(inflightInstant, table); rollbackInflightCompaction(inflightInstant, table);
// refresh table // refresh table
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true); metaClient = createMetaClient(true);
table = HoodieTable.getHoodieTable(metaClient, config, jsc); table = HoodieTable.getHoodieTable(metaClient, config, jsc);
pendingCompactionTimeline = metaClient.getActiveTimeline().filterPendingCompactionTimeline(); pendingCompactionTimeline = metaClient.getActiveTimeline().filterPendingCompactionTimeline();
} }
@@ -1253,8 +1243,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
activeTimeline.transitionCompactionRequestedToInflight(compactionInstant); activeTimeline.transitionCompactionRequestedToInflight(compactionInstant);
compactionTimer = metrics.getCompactionCtx(); compactionTimer = metrics.getCompactionCtx();
// Create a Hoodie table which encapsulated the commits and files visible // Create a Hoodie table which encapsulated the commits and files visible
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), HoodieTableMetaClient metaClient = createMetaClient(true);
config.getBasePath(), true);
HoodieTable<T> table = HoodieTable.getHoodieTable(metaClient, config, jsc); HoodieTable<T> table = HoodieTable.getHoodieTable(metaClient, config, jsc);
JavaRDD<WriteStatus> statuses = table.compact(jsc, compactionInstant.getTimestamp(), compactionPlan); JavaRDD<WriteStatus> statuses = table.compact(jsc, compactionInstant.getTimestamp(), compactionPlan);
// Force compaction action // Force compaction action
@@ -1383,7 +1372,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
try { try {
// Create a Hoodie table which encapsulated the commits and files visible // Create a Hoodie table which encapsulated the commits and files visible
HoodieTable table = HoodieTable.getHoodieTable( HoodieTable table = HoodieTable.getHoodieTable(
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config, jsc); createMetaClient(true), config, jsc);
// 0. All of the rolling stat management is only done by the DELTA commit for MOR and COMMIT for COW other wise // 0. All of the rolling stat management is only done by the DELTA commit for MOR and COMMIT for COW other wise
// there may be race conditions // there may be race conditions
HoodieRollingStatMetadata rollingStatMetadata = new HoodieRollingStatMetadata(actionType); HoodieRollingStatMetadata rollingStatMetadata = new HoodieRollingStatMetadata(actionType);

View File

@@ -0,0 +1,39 @@
/*
* 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 com.uber.hoodie.client.utils;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.config.HoodieWriteConfig;
import org.apache.spark.api.java.JavaSparkContext;
public class ClientUtils {
/**
* Create Consistency Aware MetaClient
*
* @param jsc JavaSparkContext
* @param config HoodieWriteConfig
* @param loadActiveTimelineOnLoad early loading of timeline
*/
public static HoodieTableMetaClient createMetaClient(JavaSparkContext jsc, HoodieWriteConfig config,
boolean loadActiveTimelineOnLoad) {
return new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), loadActiveTimelineOnLoad,
config.getConsistencyGuardConfig());
}
}

View File

@@ -22,6 +22,7 @@ import com.google.common.base.Preconditions;
import com.uber.hoodie.WriteStatus; import com.uber.hoodie.WriteStatus;
import com.uber.hoodie.common.model.HoodieCleaningPolicy; import com.uber.hoodie.common.model.HoodieCleaningPolicy;
import com.uber.hoodie.common.table.view.FileSystemViewStorageConfig; import com.uber.hoodie.common.table.view.FileSystemViewStorageConfig;
import com.uber.hoodie.common.util.ConsistencyGuardConfig;
import com.uber.hoodie.common.util.ReflectionUtils; import com.uber.hoodie.common.util.ReflectionUtils;
import com.uber.hoodie.index.HoodieIndex; import com.uber.hoodie.index.HoodieIndex;
import com.uber.hoodie.io.compact.strategy.CompactionStrategy; import com.uber.hoodie.io.compact.strategy.CompactionStrategy;
@@ -66,10 +67,10 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
private static final String DEFAULT_HOODIE_WRITE_STATUS_CLASS = WriteStatus.class.getName(); private static final String DEFAULT_HOODIE_WRITE_STATUS_CLASS = WriteStatus.class.getName();
private static final String FINALIZE_WRITE_PARALLELISM = "hoodie.finalize.write.parallelism"; private static final String FINALIZE_WRITE_PARALLELISM = "hoodie.finalize.write.parallelism";
private static final String DEFAULT_FINALIZE_WRITE_PARALLELISM = DEFAULT_PARALLELISM; private static final String DEFAULT_FINALIZE_WRITE_PARALLELISM = DEFAULT_PARALLELISM;
private static final String CONSISTENCY_CHECK_ENABLED_PROP = "hoodie.consistency.check.enabled";
private static final String DEFAULT_CONSISTENCY_CHECK_ENABLED = "false";
private static final String EMBEDDED_TIMELINE_SERVER_ENABLED = "hoodie.embed.timeline.server"; private static final String EMBEDDED_TIMELINE_SERVER_ENABLED = "hoodie.embed.timeline.server";
private static final String DEFAULT_EMBEDDED_TIMELINE_SERVER_ENABLED = "false"; private static final String DEFAULT_EMBEDDED_TIMELINE_SERVER_ENABLED = "false";
private static final String FAIL_ON_TIMELINE_ARCHIVING_ENABLED_PROP = "hoodie.fail.on.timeline.archiving"; private static final String FAIL_ON_TIMELINE_ARCHIVING_ENABLED_PROP = "hoodie.fail.on.timeline.archiving";
private static final String DEFAULT_FAIL_ON_TIMELINE_ARCHIVING_ENABLED = "true"; private static final String DEFAULT_FAIL_ON_TIMELINE_ARCHIVING_ENABLED = "true";
// time between successive attempts to ensure written data's metadata is consistent on storage // time between successive attempts to ensure written data's metadata is consistent on storage
@@ -85,6 +86,8 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
private static final String MAX_CONSISTENCY_CHECKS_PROP = "hoodie.consistency.check.max_checks"; private static final String MAX_CONSISTENCY_CHECKS_PROP = "hoodie.consistency.check.max_checks";
private static int DEFAULT_MAX_CONSISTENCY_CHECKS = 7; private static int DEFAULT_MAX_CONSISTENCY_CHECKS = 7;
private ConsistencyGuardConfig consistencyGuardConfig;
// Hoodie Write Client transparently rewrites File System View config when embedded mode is enabled // Hoodie Write Client transparently rewrites File System View config when embedded mode is enabled
// We keep track of original config and rewritten config // We keep track of original config and rewritten config
private final FileSystemViewStorageConfig clientSpecifiedViewStorageConfig; private final FileSystemViewStorageConfig clientSpecifiedViewStorageConfig;
@@ -94,6 +97,7 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
super(props); super(props);
Properties newProps = new Properties(); Properties newProps = new Properties();
newProps.putAll(props); newProps.putAll(props);
this.consistencyGuardConfig = ConsistencyGuardConfig.newBuilder().fromProperties(newProps).build();
this.clientSpecifiedViewStorageConfig = FileSystemViewStorageConfig.newBuilder().fromProperties(newProps).build(); this.clientSpecifiedViewStorageConfig = FileSystemViewStorageConfig.newBuilder().fromProperties(newProps).build();
this.viewStorageConfig = clientSpecifiedViewStorageConfig; this.viewStorageConfig = clientSpecifiedViewStorageConfig;
} }
@@ -162,10 +166,6 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
return Integer.parseInt(props.getProperty(FINALIZE_WRITE_PARALLELISM)); return Integer.parseInt(props.getProperty(FINALIZE_WRITE_PARALLELISM));
} }
public boolean isConsistencyCheckEnabled() {
return Boolean.parseBoolean(props.getProperty(CONSISTENCY_CHECK_ENABLED_PROP));
}
public boolean isEmbeddedTimelineServerEnabled() { public boolean isEmbeddedTimelineServerEnabled() {
return Boolean.parseBoolean(props.getProperty(EMBEDDED_TIMELINE_SERVER_ENABLED)); return Boolean.parseBoolean(props.getProperty(EMBEDDED_TIMELINE_SERVER_ENABLED));
} }
@@ -495,6 +495,14 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
return Double.valueOf(props.getProperty(HoodieMemoryConfig.WRITESTATUS_FAILURE_FRACTION_PROP)); return Double.valueOf(props.getProperty(HoodieMemoryConfig.WRITESTATUS_FAILURE_FRACTION_PROP));
} }
public ConsistencyGuardConfig getConsistencyGuardConfig() {
return consistencyGuardConfig;
}
public void setConsistencyGuardConfig(ConsistencyGuardConfig consistencyGuardConfig) {
this.consistencyGuardConfig = consistencyGuardConfig;
}
public FileSystemViewStorageConfig getViewStorageConfig() { public FileSystemViewStorageConfig getViewStorageConfig() {
return viewStorageConfig; return viewStorageConfig;
} }
@@ -520,6 +528,7 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
private boolean isMetricsConfigSet = false; private boolean isMetricsConfigSet = false;
private boolean isMemoryConfigSet = false; private boolean isMemoryConfigSet = false;
private boolean isViewConfigSet = false; private boolean isViewConfigSet = false;
private boolean isConsistencyGuardSet = false;
public Builder fromFile(File propertiesFile) throws IOException { public Builder fromFile(File propertiesFile) throws IOException {
FileReader reader = new FileReader(propertiesFile); FileReader reader = new FileReader(propertiesFile);
@@ -639,36 +648,22 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
return this; return this;
} }
public Builder withConsistencyGuardConfig(ConsistencyGuardConfig consistencyGuardConfig) {
props.putAll(consistencyGuardConfig.getProps());
isConsistencyGuardSet = true;
return this;
}
public Builder withFinalizeWriteParallelism(int parallelism) { public Builder withFinalizeWriteParallelism(int parallelism) {
props.setProperty(FINALIZE_WRITE_PARALLELISM, String.valueOf(parallelism)); props.setProperty(FINALIZE_WRITE_PARALLELISM, String.valueOf(parallelism));
return this; return this;
} }
public Builder withConsistencyCheckEnabled(boolean enabled) {
props.setProperty(CONSISTENCY_CHECK_ENABLED_PROP, String.valueOf(enabled));
return this;
}
public Builder withEmbeddedTimelineServerEnabled(boolean enabled) { public Builder withEmbeddedTimelineServerEnabled(boolean enabled) {
props.setProperty(EMBEDDED_TIMELINE_SERVER_ENABLED, String.valueOf(enabled)); props.setProperty(EMBEDDED_TIMELINE_SERVER_ENABLED, String.valueOf(enabled));
return this; return this;
} }
public Builder withInitialConsistencyCheckIntervalMs(int initialIntevalMs) {
props.setProperty(INITIAL_CONSISTENCY_CHECK_INTERVAL_MS_PROP, String.valueOf(initialIntevalMs));
return this;
}
public Builder withMaxConsistencyCheckIntervalMs(int maxIntervalMs) {
props.setProperty(MAX_CONSISTENCY_CHECK_INTERVAL_MS_PROP, String.valueOf(maxIntervalMs));
return this;
}
public Builder withMaxConsistencyChecks(int maxConsistencyChecks) {
props.setProperty(MAX_CONSISTENCY_CHECKS_PROP, String.valueOf(maxConsistencyChecks));
return this;
}
public HoodieWriteConfig build() { public HoodieWriteConfig build() {
// Check for mandatory properties // Check for mandatory properties
setDefaultOnCondition(props, !props.containsKey(INSERT_PARALLELISM), INSERT_PARALLELISM, setDefaultOnCondition(props, !props.containsKey(INSERT_PARALLELISM), INSERT_PARALLELISM,
@@ -691,8 +686,6 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
HOODIE_WRITE_STATUS_CLASS_PROP, DEFAULT_HOODIE_WRITE_STATUS_CLASS); HOODIE_WRITE_STATUS_CLASS_PROP, DEFAULT_HOODIE_WRITE_STATUS_CLASS);
setDefaultOnCondition(props, !props.containsKey(FINALIZE_WRITE_PARALLELISM), setDefaultOnCondition(props, !props.containsKey(FINALIZE_WRITE_PARALLELISM),
FINALIZE_WRITE_PARALLELISM, DEFAULT_FINALIZE_WRITE_PARALLELISM); FINALIZE_WRITE_PARALLELISM, DEFAULT_FINALIZE_WRITE_PARALLELISM);
setDefaultOnCondition(props, !props.containsKey(CONSISTENCY_CHECK_ENABLED_PROP),
CONSISTENCY_CHECK_ENABLED_PROP, DEFAULT_CONSISTENCY_CHECK_ENABLED);
setDefaultOnCondition(props, !props.containsKey(EMBEDDED_TIMELINE_SERVER_ENABLED), setDefaultOnCondition(props, !props.containsKey(EMBEDDED_TIMELINE_SERVER_ENABLED),
EMBEDDED_TIMELINE_SERVER_ENABLED, DEFAULT_EMBEDDED_TIMELINE_SERVER_ENABLED); EMBEDDED_TIMELINE_SERVER_ENABLED, DEFAULT_EMBEDDED_TIMELINE_SERVER_ENABLED);
setDefaultOnCondition(props, !props.containsKey(INITIAL_CONSISTENCY_CHECK_INTERVAL_MS_PROP), setDefaultOnCondition(props, !props.containsKey(INITIAL_CONSISTENCY_CHECK_INTERVAL_MS_PROP),
@@ -717,6 +710,8 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
HoodieMemoryConfig.newBuilder().fromProperties(props).build()); HoodieMemoryConfig.newBuilder().fromProperties(props).build());
setDefaultOnCondition(props, !isViewConfigSet, setDefaultOnCondition(props, !isViewConfigSet,
FileSystemViewStorageConfig.newBuilder().fromProperties(props).build()); FileSystemViewStorageConfig.newBuilder().fromProperties(props).build());
setDefaultOnCondition(props, !isConsistencyGuardSet,
ConsistencyGuardConfig.newBuilder().fromProperties(props).build());
// Build WriteConfig at the end // Build WriteConfig at the end
HoodieWriteConfig config = new HoodieWriteConfig(props); HoodieWriteConfig config = new HoodieWriteConfig(props);

View File

@@ -43,7 +43,6 @@ public abstract class HoodieReadHandle<T extends HoodieRecordPayload> extends Ho
return hoodieTable.getMetaClient().getFs(); return hoodieTable.getMetaClient().getFs();
} }
public Pair<String, String> getPartitionPathFilePair() { public Pair<String, String> getPartitionPathFilePair() {
return partitionPathFilePair; return partitionPathFilePair;
} }

View File

@@ -19,14 +19,11 @@
package com.uber.hoodie.io; package com.uber.hoodie.io;
import com.uber.hoodie.WriteStatus; import com.uber.hoodie.WriteStatus;
import com.uber.hoodie.common.io.storage.HoodieWrapperFileSystem;
import com.uber.hoodie.common.model.HoodieRecord; import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.model.HoodieRecordPayload; import com.uber.hoodie.common.model.HoodieRecordPayload;
import com.uber.hoodie.common.util.FSUtils; import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.common.util.FailSafeConsistencyGuard;
import com.uber.hoodie.common.util.HoodieAvroUtils; import com.uber.hoodie.common.util.HoodieAvroUtils;
import com.uber.hoodie.common.util.HoodieTimer; import com.uber.hoodie.common.util.HoodieTimer;
import com.uber.hoodie.common.util.NoOpConsistencyGuard;
import com.uber.hoodie.common.util.ReflectionUtils; import com.uber.hoodie.common.util.ReflectionUtils;
import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.exception.HoodieException; import com.uber.hoodie.exception.HoodieException;
@@ -68,13 +65,6 @@ public abstract class HoodieWriteHandle<T extends HoodieRecordPayload> extends H
config.getWriteStatusFailureFraction()); config.getWriteStatusFailureFraction());
} }
private static FileSystem getFileSystem(HoodieTable hoodieTable, HoodieWriteConfig config) {
return new HoodieWrapperFileSystem(hoodieTable.getMetaClient().getFs(), config.isConsistencyCheckEnabled()
? new FailSafeConsistencyGuard(hoodieTable.getMetaClient().getFs(),
config.getMaxConsistencyChecks(), config.getInitialConsistencyCheckIntervalMs(),
config.getMaxConsistencyCheckIntervalMs()) : new NoOpConsistencyGuard());
}
/** /**
* Generate a write token based on the currently running spark task and its place in the spark dag. * Generate a write token based on the currently running spark task and its place in the spark dag.
*/ */
@@ -175,9 +165,6 @@ public abstract class HoodieWriteHandle<T extends HoodieRecordPayload> extends H
@Override @Override
protected FileSystem getFileSystem() { protected FileSystem getFileSystem() {
return new HoodieWrapperFileSystem(hoodieTable.getMetaClient().getFs(), config.isConsistencyCheckEnabled() return hoodieTable.getMetaClient().getFs();
? new FailSafeConsistencyGuard(hoodieTable.getMetaClient().getFs(),
config.getMaxConsistencyChecks(), config.getInitialConsistencyCheckIntervalMs(),
config.getMaxConsistencyCheckIntervalMs()) : new NoOpConsistencyGuard());
} }
} }

View File

@@ -21,6 +21,7 @@ package com.uber.hoodie.table;
import com.uber.hoodie.WriteStatus; import com.uber.hoodie.WriteStatus;
import com.uber.hoodie.avro.model.HoodieCompactionPlan; import com.uber.hoodie.avro.model.HoodieCompactionPlan;
import com.uber.hoodie.avro.model.HoodieSavepointMetadata; import com.uber.hoodie.avro.model.HoodieSavepointMetadata;
import com.uber.hoodie.client.utils.ClientUtils;
import com.uber.hoodie.common.HoodieCleanStat; import com.uber.hoodie.common.HoodieCleanStat;
import com.uber.hoodie.common.HoodieRollbackStat; import com.uber.hoodie.common.HoodieRollbackStat;
import com.uber.hoodie.common.SerializableConfiguration; import com.uber.hoodie.common.SerializableConfiguration;
@@ -83,7 +84,7 @@ public abstract class HoodieTable<T extends HoodieRecordPayload> implements Seri
this.hadoopConfiguration = new SerializableConfiguration(jsc.hadoopConfiguration()); this.hadoopConfiguration = new SerializableConfiguration(jsc.hadoopConfiguration());
this.viewManager = FileSystemViewManager.createViewManager( this.viewManager = FileSystemViewManager.createViewManager(
new SerializableConfiguration(jsc.hadoopConfiguration()), config.getViewStorageConfig()); new SerializableConfiguration(jsc.hadoopConfiguration()), config.getViewStorageConfig());
this.metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true); this.metaClient = ClientUtils.createMetaClient(jsc, config, true);
this.index = HoodieIndex.createIndex(config, jsc); this.index = HoodieIndex.createIndex(config, jsc);
} }
@@ -291,7 +292,7 @@ public abstract class HoodieTable<T extends HoodieRecordPayload> implements Seri
*/ */
public void finalizeWrite(JavaSparkContext jsc, String instantTs, List<HoodieWriteStat> stats) public void finalizeWrite(JavaSparkContext jsc, String instantTs, List<HoodieWriteStat> stats)
throws HoodieIOException { throws HoodieIOException {
cleanFailedWrites(jsc, instantTs, stats, config.isConsistencyCheckEnabled()); cleanFailedWrites(jsc, instantTs, stats, config.getConsistencyGuardConfig().isConsistencyCheckEnabled());
} }
/** /**
@@ -412,7 +413,7 @@ public abstract class HoodieTable<T extends HoodieRecordPayload> implements Seri
private boolean waitForCondition(String partitionPath, Stream<Pair<String, String>> partitionFilePaths, private boolean waitForCondition(String partitionPath, Stream<Pair<String, String>> partitionFilePaths,
FileVisibility visibility) { FileVisibility visibility) {
final FileSystem fileSystem = metaClient.getFs(); final FileSystem fileSystem = metaClient.getRawFs();
List<String> fileList = partitionFilePaths.map(Pair::getValue).collect(Collectors.toList()); List<String> fileList = partitionFilePaths.map(Pair::getValue).collect(Collectors.toList());
try { try {
getFailSafeConsistencyGuard(fileSystem).waitTill(partitionPath, fileList, visibility); getFailSafeConsistencyGuard(fileSystem).waitTill(partitionPath, fileList, visibility);
@@ -424,8 +425,6 @@ public abstract class HoodieTable<T extends HoodieRecordPayload> implements Seri
} }
private ConsistencyGuard getFailSafeConsistencyGuard(FileSystem fileSystem) { private ConsistencyGuard getFailSafeConsistencyGuard(FileSystem fileSystem) {
return new FailSafeConsistencyGuard(fileSystem, config.getMaxConsistencyChecks(), return new FailSafeConsistencyGuard(fileSystem, config.getConsistencyGuardConfig());
config.getInitialConsistencyCheckIntervalMs(),
config.getMaxConsistencyCheckIntervalMs());
} }
} }

View File

@@ -48,6 +48,7 @@ import com.uber.hoodie.common.table.timeline.HoodieInstant;
import com.uber.hoodie.common.table.timeline.HoodieInstant.State; import com.uber.hoodie.common.table.timeline.HoodieInstant.State;
import com.uber.hoodie.common.util.AvroUtils; import com.uber.hoodie.common.util.AvroUtils;
import com.uber.hoodie.common.util.CompactionUtils; import com.uber.hoodie.common.util.CompactionUtils;
import com.uber.hoodie.common.util.ConsistencyGuardConfig;
import com.uber.hoodie.common.util.FSUtils; import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.common.util.collection.Pair; import com.uber.hoodie.common.util.collection.Pair;
import com.uber.hoodie.config.HoodieCompactionConfig; import com.uber.hoodie.config.HoodieCompactionConfig;
@@ -195,7 +196,8 @@ public class TestCleaner extends TestHoodieClientBase {
HoodieCompactionConfig.newBuilder().withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS) HoodieCompactionConfig.newBuilder().withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS)
.retainFileVersions(maxVersions).build()) .retainFileVersions(maxVersions).build())
.withParallelism(1, 1).withBulkInsertParallelism(1) .withParallelism(1, 1).withBulkInsertParallelism(1)
.withFinalizeWriteParallelism(1).withConsistencyCheckEnabled(true) .withFinalizeWriteParallelism(1)
.withConsistencyGuardConfig(ConsistencyGuardConfig.newBuilder().withConsistencyCheckEnabled(true).build())
.build(); .build();
HoodieWriteClient client = getHoodieWriteClient(cfg); HoodieWriteClient client = getHoodieWriteClient(cfg);
@@ -357,7 +359,9 @@ public class TestCleaner extends TestHoodieClientBase {
HoodieCompactionConfig.newBuilder() HoodieCompactionConfig.newBuilder()
.withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS).retainCommits(maxCommits).build()) .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS).retainCommits(maxCommits).build())
.withParallelism(1, 1).withBulkInsertParallelism(1) .withParallelism(1, 1).withBulkInsertParallelism(1)
.withFinalizeWriteParallelism(1).withConsistencyCheckEnabled(true).build(); .withFinalizeWriteParallelism(1)
.withConsistencyGuardConfig(ConsistencyGuardConfig.newBuilder().withConsistencyCheckEnabled(true).build())
.build();
HoodieWriteClient client = getHoodieWriteClient(cfg); HoodieWriteClient client = getHoodieWriteClient(cfg);
final Function2<List<HoodieRecord>, String, Integer> recordInsertGenWrappedFunction = final Function2<List<HoodieRecord>, String, Integer> recordInsertGenWrappedFunction =

View File

@@ -20,6 +20,7 @@ package com.uber.hoodie;
import com.uber.hoodie.common.HoodieClientTestUtils; import com.uber.hoodie.common.HoodieClientTestUtils;
import com.uber.hoodie.common.util.ConsistencyGuard; import com.uber.hoodie.common.util.ConsistencyGuard;
import com.uber.hoodie.common.util.ConsistencyGuardConfig;
import com.uber.hoodie.common.util.FSUtils; import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.common.util.FailSafeConsistencyGuard; import com.uber.hoodie.common.util.FailSafeConsistencyGuard;
import java.io.IOException; import java.io.IOException;
@@ -58,7 +59,7 @@ public class TestConsistencyGuard {
HoodieClientTestUtils.fakeDataFile(basePath, "partition/path", "000", "f2"); HoodieClientTestUtils.fakeDataFile(basePath, "partition/path", "000", "f2");
HoodieClientTestUtils.fakeDataFile(basePath, "partition/path", "000", "f3"); HoodieClientTestUtils.fakeDataFile(basePath, "partition/path", "000", "f3");
ConsistencyGuard passing = new FailSafeConsistencyGuard(fs, 1, 1000, 1000); ConsistencyGuard passing = new FailSafeConsistencyGuard(fs, getConsistencyGuardConfig(1, 1000, 1000));
passing.waitTillFileAppears(new Path(basePath + "/partition/path/f1_1-0-1_000.parquet")); passing.waitTillFileAppears(new Path(basePath + "/partition/path/f1_1-0-1_000.parquet"));
passing.waitTillFileAppears(new Path(basePath + "/partition/path/f2_1-0-1_000.parquet")); passing.waitTillFileAppears(new Path(basePath + "/partition/path/f2_1-0-1_000.parquet"));
passing.waitTillAllFilesAppear(basePath + "/partition/path", passing.waitTillAllFilesAppear(basePath + "/partition/path",
@@ -77,7 +78,7 @@ public class TestConsistencyGuard {
@Test(expected = TimeoutException.class) @Test(expected = TimeoutException.class)
public void testCheckFailingAppear() throws Exception { public void testCheckFailingAppear() throws Exception {
HoodieClientTestUtils.fakeDataFile(basePath, "partition/path", "000", "f1"); HoodieClientTestUtils.fakeDataFile(basePath, "partition/path", "000", "f1");
ConsistencyGuard passing = new FailSafeConsistencyGuard(fs, 3, 10, 10); ConsistencyGuard passing = new FailSafeConsistencyGuard(fs, getConsistencyGuardConfig());
passing.waitTillAllFilesAppear(basePath + "/partition/path", passing.waitTillAllFilesAppear(basePath + "/partition/path",
Arrays.asList(basePath + "/partition/path/f1_1-0-2_000.parquet", Arrays.asList(basePath + "/partition/path/f1_1-0-2_000.parquet",
basePath + "/partition/path/f2_1-0-2_000.parquet")); basePath + "/partition/path/f2_1-0-2_000.parquet"));
@@ -87,14 +88,14 @@ public class TestConsistencyGuard {
@Test(expected = TimeoutException.class) @Test(expected = TimeoutException.class)
public void testCheckFailingAppears() throws Exception { public void testCheckFailingAppears() throws Exception {
HoodieClientTestUtils.fakeDataFile(basePath, "partition/path", "000", "f1"); HoodieClientTestUtils.fakeDataFile(basePath, "partition/path", "000", "f1");
ConsistencyGuard passing = new FailSafeConsistencyGuard(fs, 3, 10, 10); ConsistencyGuard passing = new FailSafeConsistencyGuard(fs, getConsistencyGuardConfig());
passing.waitTillFileAppears(new Path(basePath + "/partition/path/f1_1-0-2_000.parquet")); passing.waitTillFileAppears(new Path(basePath + "/partition/path/f1_1-0-2_000.parquet"));
} }
@Test(expected = TimeoutException.class) @Test(expected = TimeoutException.class)
public void testCheckFailingDisappear() throws Exception { public void testCheckFailingDisappear() throws Exception {
HoodieClientTestUtils.fakeDataFile(basePath, "partition/path", "000", "f1"); HoodieClientTestUtils.fakeDataFile(basePath, "partition/path", "000", "f1");
ConsistencyGuard passing = new FailSafeConsistencyGuard(fs, 3, 10, 10); ConsistencyGuard passing = new FailSafeConsistencyGuard(fs, getConsistencyGuardConfig());
passing.waitTillAllFilesDisappear(basePath + "/partition/path", passing.waitTillAllFilesDisappear(basePath + "/partition/path",
Arrays.asList(basePath + "/partition/path/f1_1-0-1_000.parquet", Arrays.asList(basePath + "/partition/path/f1_1-0-1_000.parquet",
basePath + "/partition/path/f2_1-0-2_000.parquet")); basePath + "/partition/path/f2_1-0-2_000.parquet"));
@@ -104,7 +105,17 @@ public class TestConsistencyGuard {
public void testCheckFailingDisappears() throws Exception { public void testCheckFailingDisappears() throws Exception {
HoodieClientTestUtils.fakeDataFile(basePath, "partition/path", "000", "f1"); HoodieClientTestUtils.fakeDataFile(basePath, "partition/path", "000", "f1");
HoodieClientTestUtils.fakeDataFile(basePath, "partition/path", "000", "f1"); HoodieClientTestUtils.fakeDataFile(basePath, "partition/path", "000", "f1");
ConsistencyGuard passing = new FailSafeConsistencyGuard(fs, 3, 10, 10); ConsistencyGuard passing = new FailSafeConsistencyGuard(fs, getConsistencyGuardConfig());
passing.waitTillFileDisappears(new Path(basePath + "/partition/path/f1_1-0-1_000.parquet")); passing.waitTillFileDisappears(new Path(basePath + "/partition/path/f1_1-0-1_000.parquet"));
} }
private ConsistencyGuardConfig getConsistencyGuardConfig() {
return getConsistencyGuardConfig(3, 10, 10);
}
private ConsistencyGuardConfig getConsistencyGuardConfig(int maxChecks, int initalSleep, int maxSleep) {
return ConsistencyGuardConfig.newBuilder().withConsistencyCheckEnabled(true)
.withInitialConsistencyCheckIntervalMs(initalSleep).withMaxConsistencyCheckIntervalMs(maxSleep)
.withMaxConsistencyChecks(maxChecks).build();
}
} }

View File

@@ -26,7 +26,7 @@ import static org.junit.Assert.assertTrue;
import com.uber.hoodie.common.HoodieCleanStat; import com.uber.hoodie.common.HoodieCleanStat;
import com.uber.hoodie.common.HoodieClientTestUtils; import com.uber.hoodie.common.HoodieClientTestUtils;
import com.uber.hoodie.common.HoodieTestDataGenerator; import com.uber.hoodie.common.HoodieTestDataGenerator;
import com.uber.hoodie.common.TestRawTripPayload; import com.uber.hoodie.common.TestRawTripPayload.MetadataMergeWriteStatus;
import com.uber.hoodie.common.model.HoodiePartitionMetadata; import com.uber.hoodie.common.model.HoodiePartitionMetadata;
import com.uber.hoodie.common.model.HoodieRecord; import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.model.HoodieTableType; import com.uber.hoodie.common.model.HoodieTableType;
@@ -37,12 +37,14 @@ import com.uber.hoodie.common.table.SyncableFileSystemView;
import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline; import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
import com.uber.hoodie.common.table.view.FileSystemViewStorageConfig; import com.uber.hoodie.common.table.view.FileSystemViewStorageConfig;
import com.uber.hoodie.common.table.view.FileSystemViewStorageType; import com.uber.hoodie.common.table.view.FileSystemViewStorageType;
import com.uber.hoodie.common.util.ConsistencyGuardConfig;
import com.uber.hoodie.common.util.FSUtils; import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.config.HoodieCompactionConfig; import com.uber.hoodie.config.HoodieCompactionConfig;
import com.uber.hoodie.config.HoodieIndexConfig; import com.uber.hoodie.config.HoodieIndexConfig;
import com.uber.hoodie.config.HoodieStorageConfig; import com.uber.hoodie.config.HoodieStorageConfig;
import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.index.HoodieIndex; import com.uber.hoodie.index.HoodieIndex;
import com.uber.hoodie.index.HoodieIndex.IndexType;
import com.uber.hoodie.table.HoodieTable; import com.uber.hoodie.table.HoodieTable;
import java.io.File; import java.io.File;
import java.io.IOException; import java.io.IOException;
@@ -191,12 +193,12 @@ public class TestHoodieClientBase implements Serializable {
return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA) return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA)
.withParallelism(2, 2) .withParallelism(2, 2)
.withBulkInsertParallelism(2).withFinalizeWriteParallelism(2) .withBulkInsertParallelism(2).withFinalizeWriteParallelism(2)
.withWriteStatusClass(TestRawTripPayload.MetadataMergeWriteStatus.class) .withWriteStatusClass(MetadataMergeWriteStatus.class)
.withConsistencyCheckEnabled(true) .withConsistencyGuardConfig(ConsistencyGuardConfig.newBuilder().withConsistencyCheckEnabled(true).build())
.withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024).build()) .withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024).build())
.withStorageConfig(HoodieStorageConfig.newBuilder().limitFileSize(1024 * 1024).build()) .withStorageConfig(HoodieStorageConfig.newBuilder().limitFileSize(1024 * 1024).build())
.forTable("test-trip-table") .forTable("test-trip-table")
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()) .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(IndexType.BLOOM).build())
.withEmbeddedTimelineServerEnabled(true).withFileSystemViewConfig( .withEmbeddedTimelineServerEnabled(true).withFileSystemViewConfig(
FileSystemViewStorageConfig.newBuilder().withStorageType(FileSystemViewStorageType.EMBEDDED_KV_STORE) FileSystemViewStorageConfig.newBuilder().withStorageType(FileSystemViewStorageType.EMBEDDED_KV_STORE)
.build()); .build());

View File

@@ -39,6 +39,7 @@ import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline; import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.table.TableFileSystemView; import com.uber.hoodie.common.table.TableFileSystemView;
import com.uber.hoodie.common.table.timeline.HoodieInstant; import com.uber.hoodie.common.table.timeline.HoodieInstant;
import com.uber.hoodie.common.util.ConsistencyGuardConfig;
import com.uber.hoodie.common.util.FSUtils; import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.common.util.ParquetUtils; import com.uber.hoodie.common.util.ParquetUtils;
import com.uber.hoodie.common.util.collection.Pair; import com.uber.hoodie.common.util.collection.Pair;
@@ -686,8 +687,13 @@ public class TestHoodieClientOnCopyOnWriteStorage extends TestHoodieClientBase {
private Pair<Path, JavaRDD<WriteStatus>> testConsistencyCheck(HoodieTableMetaClient metaClient, String commitTime) private Pair<Path, JavaRDD<WriteStatus>> testConsistencyCheck(HoodieTableMetaClient metaClient, String commitTime)
throws Exception { throws Exception {
HoodieWriteConfig cfg = getConfigBuilder().withAutoCommit(false).withMaxConsistencyCheckIntervalMs(1) HoodieWriteConfig cfg = getConfigBuilder().withAutoCommit(false)
.withInitialConsistencyCheckIntervalMs(1).build(); .withConsistencyGuardConfig(ConsistencyGuardConfig.newBuilder()
.withConsistencyCheckEnabled(true)
.withMaxConsistencyCheckIntervalMs(1)
.withInitialConsistencyCheckIntervalMs(1)
.build())
.build();
HoodieWriteClient client = getHoodieWriteClient(cfg); HoodieWriteClient client = getHoodieWriteClient(cfg);
client.startCommitWithTime(commitTime); client.startCommitWithTime(commitTime);

View File

@@ -22,6 +22,7 @@ import com.uber.hoodie.common.storage.StorageSchemes;
import com.uber.hoodie.common.util.ConsistencyGuard; import com.uber.hoodie.common.util.ConsistencyGuard;
import com.uber.hoodie.common.util.FSUtils; import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.common.util.NoOpConsistencyGuard; import com.uber.hoodie.common.util.NoOpConsistencyGuard;
import com.uber.hoodie.exception.HoodieException;
import com.uber.hoodie.exception.HoodieIOException; import com.uber.hoodie.exception.HoodieIOException;
import java.io.IOException; import java.io.IOException;
import java.net.URI; import java.net.URI;
@@ -236,7 +237,28 @@ public class HoodieWrapperFileSystem extends FileSystem {
@Override @Override
public boolean rename(Path src, Path dst) throws IOException { public boolean rename(Path src, Path dst) throws IOException {
return fileSystem.rename(convertToDefaultPath(src), convertToDefaultPath(dst)); try {
consistencyGuard.waitTillFileAppears(convertToDefaultPath(src));
} catch (TimeoutException e) {
throw new HoodieException("Timed out waiting for " + src + " to appear", e);
}
boolean success = fileSystem.rename(convertToDefaultPath(src), convertToDefaultPath(dst));
if (success) {
try {
consistencyGuard.waitTillFileAppears(convertToDefaultPath(dst));
} catch (TimeoutException e) {
throw new HoodieException("Timed out waiting for " + dst + " to appear", e);
}
try {
consistencyGuard.waitTillFileDisappears(convertToDefaultPath(src));
} catch (TimeoutException e) {
throw new HoodieException("Timed out waiting for " + src + " to disappear", e);
}
}
return success;
} }
@Override @Override
@@ -247,7 +269,7 @@ public class HoodieWrapperFileSystem extends FileSystem {
try { try {
consistencyGuard.waitTillFileDisappears(f); consistencyGuard.waitTillFileDisappears(f);
} catch (TimeoutException e) { } catch (TimeoutException e) {
return false; throw new HoodieException("Timed out waiting for " + f + " to disappear", e);
} }
} }
return success; return success;
@@ -270,7 +292,15 @@ public class HoodieWrapperFileSystem extends FileSystem {
@Override @Override
public boolean mkdirs(Path f, FsPermission permission) throws IOException { public boolean mkdirs(Path f, FsPermission permission) throws IOException {
return fileSystem.mkdirs(convertToDefaultPath(f), permission); boolean success = fileSystem.mkdirs(convertToDefaultPath(f), permission);
if (success) {
try {
consistencyGuard.waitTillFileAppears(convertToDefaultPath(f));
} catch (TimeoutException e) {
throw new HoodieException("Timed out waiting for directory " + f + " to appear", e);
}
}
return success;
} }
@Override @Override
@@ -353,31 +383,39 @@ public class HoodieWrapperFileSystem extends FileSystem {
@Override @Override
public FSDataOutputStream createNonRecursive(Path f, boolean overwrite, int bufferSize, public FSDataOutputStream createNonRecursive(Path f, boolean overwrite, int bufferSize,
short replication, long blockSize, Progressable progress) throws IOException { short replication, long blockSize, Progressable progress) throws IOException {
return fileSystem Path p = convertToDefaultPath(f);
.createNonRecursive(convertToDefaultPath(f), overwrite, bufferSize, replication, blockSize, return wrapOutputStream(p, fileSystem.createNonRecursive(p, overwrite, bufferSize, replication, blockSize,
progress); progress));
} }
@Override @Override
public FSDataOutputStream createNonRecursive(Path f, FsPermission permission, boolean overwrite, public FSDataOutputStream createNonRecursive(Path f, FsPermission permission, boolean overwrite,
int bufferSize, short replication, long blockSize, Progressable progress) throws IOException { int bufferSize, short replication, long blockSize, Progressable progress) throws IOException {
return fileSystem Path p = convertToDefaultPath(f);
.createNonRecursive(convertToDefaultPath(f), permission, overwrite, bufferSize, replication, return wrapOutputStream(p, fileSystem.createNonRecursive(p, permission, overwrite, bufferSize, replication,
blockSize, progress); blockSize, progress));
} }
@Override @Override
public FSDataOutputStream createNonRecursive(Path f, FsPermission permission, public FSDataOutputStream createNonRecursive(Path f, FsPermission permission,
EnumSet<CreateFlag> flags, int bufferSize, short replication, long blockSize, EnumSet<CreateFlag> flags, int bufferSize, short replication, long blockSize,
Progressable progress) throws IOException { Progressable progress) throws IOException {
return fileSystem Path p = convertToDefaultPath(f);
.createNonRecursive(convertToDefaultPath(f), permission, flags, bufferSize, replication, return wrapOutputStream(p, fileSystem.createNonRecursive(p, permission, flags, bufferSize, replication,
blockSize, progress); blockSize, progress));
} }
@Override @Override
public boolean createNewFile(Path f) throws IOException { public boolean createNewFile(Path f) throws IOException {
return fileSystem.createNewFile(convertToDefaultPath(f)); boolean newFile = fileSystem.createNewFile(convertToDefaultPath(f));
if (newFile) {
try {
consistencyGuard.waitTillFileAppears(convertToDefaultPath(f));
} catch (TimeoutException e) {
throw new HoodieException("Timed out waiting for " + f + " to appear", e);
}
}
return newFile;
} }
@Override @Override
@@ -394,6 +432,11 @@ public class HoodieWrapperFileSystem extends FileSystem {
public void concat(Path trg, Path[] psrcs) throws IOException { public void concat(Path trg, Path[] psrcs) throws IOException {
Path[] psrcsNew = convertDefaults(psrcs); Path[] psrcsNew = convertDefaults(psrcs);
fileSystem.concat(convertToDefaultPath(trg), psrcsNew); fileSystem.concat(convertToDefaultPath(trg), psrcsNew);
try {
consistencyGuard.waitTillFileAppears(convertToDefaultPath(trg));
} catch (TimeoutException e) {
throw new HoodieException("Timed out waiting for " + trg + " to appear", e);
}
} }
@Override @Override
@@ -408,7 +451,7 @@ public class HoodieWrapperFileSystem extends FileSystem {
@Override @Override
public boolean delete(Path f) throws IOException { public boolean delete(Path f) throws IOException {
return fileSystem.delete(convertToDefaultPath(f)); return delete(f, true);
} }
@Override @Override
@@ -493,62 +536,100 @@ public class HoodieWrapperFileSystem extends FileSystem {
@Override @Override
public boolean mkdirs(Path f) throws IOException { public boolean mkdirs(Path f) throws IOException {
return fileSystem.mkdirs(convertToDefaultPath(f)); boolean success = fileSystem.mkdirs(convertToDefaultPath(f));
if (success) {
try {
consistencyGuard.waitTillFileAppears(convertToDefaultPath(f));
} catch (TimeoutException e) {
throw new HoodieException("Timed out waiting for directory " + f + " to appear", e);
}
}
return success;
} }
@Override @Override
public void copyFromLocalFile(Path src, Path dst) throws IOException { public void copyFromLocalFile(Path src, Path dst) throws IOException {
fileSystem.copyFromLocalFile(convertToDefaultPath(src), convertToDefaultPath(dst)); fileSystem.copyFromLocalFile(convertToLocalPath(src), convertToDefaultPath(dst));
try {
consistencyGuard.waitTillFileAppears(convertToDefaultPath(dst));
} catch (TimeoutException e) {
throw new HoodieException("Timed out waiting for destination " + dst + " to appear", e);
}
} }
@Override @Override
public void moveFromLocalFile(Path[] srcs, Path dst) throws IOException { public void moveFromLocalFile(Path[] srcs, Path dst) throws IOException {
fileSystem.moveFromLocalFile(convertDefaults(srcs), convertToDefaultPath(dst)); fileSystem.moveFromLocalFile(convertLocalPaths(srcs), convertToDefaultPath(dst));
try {
consistencyGuard.waitTillFileAppears(convertToDefaultPath(dst));
} catch (TimeoutException e) {
throw new HoodieException("Timed out waiting for destination " + dst + " to appear", e);
}
} }
@Override @Override
public void moveFromLocalFile(Path src, Path dst) throws IOException { public void moveFromLocalFile(Path src, Path dst) throws IOException {
fileSystem.moveFromLocalFile(convertToDefaultPath(src), convertToDefaultPath(dst)); fileSystem.moveFromLocalFile(convertToLocalPath(src), convertToDefaultPath(dst));
try {
consistencyGuard.waitTillFileAppears(convertToDefaultPath(dst));
} catch (TimeoutException e) {
throw new HoodieException("Timed out waiting for destination " + dst + " to appear", e);
}
} }
@Override @Override
public void copyFromLocalFile(boolean delSrc, Path src, Path dst) throws IOException { public void copyFromLocalFile(boolean delSrc, Path src, Path dst) throws IOException {
fileSystem.copyFromLocalFile(delSrc, convertToDefaultPath(src), convertToDefaultPath(dst)); fileSystem.copyFromLocalFile(delSrc, convertToLocalPath(src), convertToDefaultPath(dst));
try {
consistencyGuard.waitTillFileAppears(convertToDefaultPath(dst));
} catch (TimeoutException e) {
throw new HoodieException("Timed out waiting for destination " + dst + " to appear", e);
}
} }
@Override @Override
public void copyFromLocalFile(boolean delSrc, boolean overwrite, Path[] srcs, Path dst) public void copyFromLocalFile(boolean delSrc, boolean overwrite, Path[] srcs, Path dst)
throws IOException { throws IOException {
fileSystem fileSystem
.copyFromLocalFile(delSrc, overwrite, convertDefaults(srcs), convertToDefaultPath(dst)); .copyFromLocalFile(delSrc, overwrite, convertLocalPaths(srcs), convertToDefaultPath(dst));
try {
consistencyGuard.waitTillFileAppears(convertToDefaultPath(dst));
} catch (TimeoutException e) {
throw new HoodieException("Timed out waiting for destination " + dst + " to appear", e);
}
} }
@Override @Override
public void copyFromLocalFile(boolean delSrc, boolean overwrite, Path src, Path dst) public void copyFromLocalFile(boolean delSrc, boolean overwrite, Path src, Path dst)
throws IOException { throws IOException {
fileSystem fileSystem
.copyFromLocalFile(delSrc, overwrite, convertToDefaultPath(src), convertToDefaultPath(dst)); .copyFromLocalFile(delSrc, overwrite, convertToLocalPath(src), convertToDefaultPath(dst));
try {
consistencyGuard.waitTillFileAppears(convertToDefaultPath(dst));
} catch (TimeoutException e) {
throw new HoodieException("Timed out waiting for destination " + dst + " to appear", e);
}
} }
@Override @Override
public void copyToLocalFile(Path src, Path dst) throws IOException { public void copyToLocalFile(Path src, Path dst) throws IOException {
fileSystem.copyToLocalFile(convertToDefaultPath(src), convertToDefaultPath(dst)); fileSystem.copyToLocalFile(convertToDefaultPath(src), convertToLocalPath(dst));
} }
@Override @Override
public void moveToLocalFile(Path src, Path dst) throws IOException { public void moveToLocalFile(Path src, Path dst) throws IOException {
fileSystem.moveToLocalFile(convertToDefaultPath(src), convertToDefaultPath(dst)); fileSystem.moveToLocalFile(convertToDefaultPath(src), convertToLocalPath(dst));
} }
@Override @Override
public void copyToLocalFile(boolean delSrc, Path src, Path dst) throws IOException { public void copyToLocalFile(boolean delSrc, Path src, Path dst) throws IOException {
fileSystem.copyToLocalFile(delSrc, convertToDefaultPath(src), convertToDefaultPath(dst)); fileSystem.copyToLocalFile(delSrc, convertToDefaultPath(src), convertToLocalPath(dst));
} }
@Override @Override
public void copyToLocalFile(boolean delSrc, Path src, Path dst, boolean useRawLocalFileSystem) public void copyToLocalFile(boolean delSrc, Path src, Path dst, boolean useRawLocalFileSystem)
throws IOException { throws IOException {
fileSystem.copyToLocalFile(delSrc, convertToDefaultPath(src), convertToDefaultPath(dst), fileSystem.copyToLocalFile(delSrc, convertToDefaultPath(src), convertToLocalPath(dst),
useRawLocalFileSystem); useRawLocalFileSystem);
} }
@@ -787,6 +868,22 @@ public class HoodieWrapperFileSystem extends FileSystem {
return convertPathWithScheme(oldPath, fileSystem.getScheme()); return convertPathWithScheme(oldPath, fileSystem.getScheme());
} }
private Path convertToLocalPath(Path oldPath) {
try {
return convertPathWithScheme(oldPath, FileSystem.getLocal(getConf()).getScheme());
} catch (IOException e) {
throw new HoodieIOException(e.getMessage(), e);
}
}
private Path[] convertLocalPaths(Path[] psrcs) {
Path[] psrcsNew = new Path[psrcs.length];
for (int i = 0; i < psrcs.length; i++) {
psrcsNew[i] = convertToLocalPath(psrcs[i]);
}
return psrcsNew;
}
private Path[] convertDefaults(Path[] psrcs) { private Path[] convertDefaults(Path[] psrcs) {
Path[] psrcsNew = new Path[psrcs.length]; Path[] psrcsNew = new Path[psrcs.length];
for (int i = 0; i < psrcs.length; i++) { for (int i = 0; i < psrcs.length; i++) {
@@ -803,4 +900,8 @@ public class HoodieWrapperFileSystem extends FileSystem {
throw new IllegalArgumentException(file.toString() throw new IllegalArgumentException(file.toString()
+ " does not have a open stream. Cannot get the bytes written on the stream"); + " does not have a open stream. Cannot get the bytes written on the stream");
} }
public FileSystem getFileSystem() {
return fileSystem;
}
} }

View File

@@ -20,12 +20,17 @@ package com.uber.hoodie.common.table;
import static com.uber.hoodie.common.model.HoodieTableType.MERGE_ON_READ; import static com.uber.hoodie.common.model.HoodieTableType.MERGE_ON_READ;
import com.google.common.base.Preconditions;
import com.uber.hoodie.common.SerializableConfiguration; import com.uber.hoodie.common.SerializableConfiguration;
import com.uber.hoodie.common.io.storage.HoodieWrapperFileSystem;
import com.uber.hoodie.common.model.HoodieTableType; import com.uber.hoodie.common.model.HoodieTableType;
import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline; import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
import com.uber.hoodie.common.table.timeline.HoodieArchivedTimeline; import com.uber.hoodie.common.table.timeline.HoodieArchivedTimeline;
import com.uber.hoodie.common.table.timeline.HoodieInstant; import com.uber.hoodie.common.table.timeline.HoodieInstant;
import com.uber.hoodie.common.util.ConsistencyGuardConfig;
import com.uber.hoodie.common.util.FSUtils; import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.common.util.FailSafeConsistencyGuard;
import com.uber.hoodie.common.util.NoOpConsistencyGuard;
import com.uber.hoodie.exception.DatasetNotFoundException; import com.uber.hoodie.exception.DatasetNotFoundException;
import com.uber.hoodie.exception.HoodieException; import com.uber.hoodie.exception.HoodieException;
import java.io.File; import java.io.File;
@@ -66,13 +71,14 @@ public class HoodieTableMetaClient implements Serializable {
public static final String MARKER_EXTN = ".marker"; public static final String MARKER_EXTN = ".marker";
private String basePath; private String basePath;
private transient FileSystem fs; private transient HoodieWrapperFileSystem fs;
private String metaPath; private String metaPath;
private SerializableConfiguration hadoopConf; private SerializableConfiguration hadoopConf;
private HoodieTableType tableType; private HoodieTableType tableType;
private HoodieTableConfig tableConfig; private HoodieTableConfig tableConfig;
private HoodieActiveTimeline activeTimeline; private HoodieActiveTimeline activeTimeline;
private HoodieArchivedTimeline archivedTimeline; private HoodieArchivedTimeline archivedTimeline;
private ConsistencyGuardConfig consistencyGuardConfig = ConsistencyGuardConfig.newBuilder().build();
public HoodieTableMetaClient(Configuration conf, String basePath) public HoodieTableMetaClient(Configuration conf, String basePath)
throws DatasetNotFoundException { throws DatasetNotFoundException {
@@ -81,13 +87,19 @@ public class HoodieTableMetaClient implements Serializable {
} }
public HoodieTableMetaClient(Configuration conf, String basePath, public HoodieTableMetaClient(Configuration conf, String basePath,
boolean loadActiveTimelineOnLoad) boolean loadActiveTimelineOnLoad) {
this(conf, basePath, loadActiveTimelineOnLoad, ConsistencyGuardConfig.newBuilder().build());
}
public HoodieTableMetaClient(Configuration conf, String basePath,
boolean loadActiveTimelineOnLoad, ConsistencyGuardConfig consistencyGuardConfig)
throws DatasetNotFoundException { throws DatasetNotFoundException {
log.info("Loading HoodieTableMetaClient from " + basePath); log.info("Loading HoodieTableMetaClient from " + basePath);
this.basePath = basePath; this.basePath = basePath;
this.consistencyGuardConfig = consistencyGuardConfig;
this.hadoopConf = new SerializableConfiguration(conf); this.hadoopConf = new SerializableConfiguration(conf);
Path basePathDir = new Path(this.basePath); Path basePathDir = new Path(this.basePath);
this.metaPath = basePath + File.separator + METAFOLDER_NAME; this.metaPath = new Path(basePath, METAFOLDER_NAME).toString();
Path metaPathDir = new Path(this.metaPath); Path metaPathDir = new Path(this.metaPath);
this.fs = getFs(); this.fs = getFs();
DatasetNotFoundException.checkValidDataset(fs, basePathDir, metaPathDir); DatasetNotFoundException.checkValidDataset(fs, basePathDir, metaPathDir);
@@ -190,13 +202,25 @@ public class HoodieTableMetaClient implements Serializable {
/** /**
* Get the FS implementation for this table * Get the FS implementation for this table
*/ */
public FileSystem getFs() { public HoodieWrapperFileSystem getFs() {
if (fs == null) { if (fs == null) {
fs = FSUtils.getFs(metaPath, hadoopConf.get()); FileSystem fileSystem = FSUtils.getFs(metaPath, hadoopConf.get());
Preconditions.checkArgument(!(fileSystem instanceof HoodieWrapperFileSystem),
"File System not expected to be that of HoodieWrapperFileSystem");
fs = new HoodieWrapperFileSystem(fileSystem, consistencyGuardConfig.isConsistencyCheckEnabled()
? new FailSafeConsistencyGuard(fileSystem, consistencyGuardConfig) : new NoOpConsistencyGuard());
} }
return fs; return fs;
} }
/**
* Return raw file-system
* @return
*/
public FileSystem getRawFs() {
return getFs().getFileSystem();
}
public Configuration getHadoopConf() { public Configuration getHadoopConf() {
return hadoopConf.get(); return hadoopConf.get();
} }
@@ -223,6 +247,10 @@ public class HoodieTableMetaClient implements Serializable {
return activeTimeline; return activeTimeline;
} }
public ConsistencyGuardConfig getConsistencyGuardConfig() {
return consistencyGuardConfig;
}
/** /**
* Get the archived commits as a timeline. This is costly operation, as all data from the archived * Get the archived commits as a timeline. This is costly operation, as all data from the archived
* files are read. This should not be used, unless for historical debugging purposes * files are read. This should not be used, unless for historical debugging purposes

View File

@@ -0,0 +1,121 @@
/*
* 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 com.uber.hoodie.common.util;
import com.uber.hoodie.config.DefaultHoodieConfig;
import java.io.File;
import java.io.FileReader;
import java.io.IOException;
import java.util.Properties;
public class ConsistencyGuardConfig extends DefaultHoodieConfig {
private static final String CONSISTENCY_CHECK_ENABLED_PROP = "hoodie.consistency.check.enabled";
private static final String DEFAULT_CONSISTENCY_CHECK_ENABLED = "false";
// time between successive attempts to ensure written data's metadata is consistent on storage
private static final String INITIAL_CONSISTENCY_CHECK_INTERVAL_MS_PROP =
"hoodie.consistency.check.initial_interval_ms";
private static long DEFAULT_INITIAL_CONSISTENCY_CHECK_INTERVAL_MS = 2000L;
// max interval time
private static final String MAX_CONSISTENCY_CHECK_INTERVAL_MS_PROP = "hoodie.consistency.check.max_interval_ms";
private static long DEFAULT_MAX_CONSISTENCY_CHECK_INTERVAL_MS = 300000L;
// maximum number of checks, for consistency of written data. Will wait upto 256 Secs
private static final String MAX_CONSISTENCY_CHECKS_PROP = "hoodie.consistency.check.max_checks";
private static int DEFAULT_MAX_CONSISTENCY_CHECKS = 7;
public ConsistencyGuardConfig(Properties props) {
super(props);
}
public static ConsistencyGuardConfig.Builder newBuilder() {
return new Builder();
}
public boolean isConsistencyCheckEnabled() {
return Boolean.parseBoolean(props.getProperty(CONSISTENCY_CHECK_ENABLED_PROP));
}
public int getMaxConsistencyChecks() {
return Integer.parseInt(props.getProperty(MAX_CONSISTENCY_CHECKS_PROP));
}
public int getInitialConsistencyCheckIntervalMs() {
return Integer.parseInt(props.getProperty(INITIAL_CONSISTENCY_CHECK_INTERVAL_MS_PROP));
}
public int getMaxConsistencyCheckIntervalMs() {
return Integer.parseInt(props.getProperty(MAX_CONSISTENCY_CHECK_INTERVAL_MS_PROP));
}
public static class Builder {
private final Properties props = new Properties();
public Builder fromFile(File propertiesFile) throws IOException {
FileReader reader = new FileReader(propertiesFile);
try {
props.load(reader);
return this;
} finally {
reader.close();
}
}
public Builder fromProperties(Properties props) {
this.props.putAll(props);
return this;
}
public Builder withConsistencyCheckEnabled(boolean enabled) {
props.setProperty(CONSISTENCY_CHECK_ENABLED_PROP, String.valueOf(enabled));
return this;
}
public Builder withInitialConsistencyCheckIntervalMs(int initialIntevalMs) {
props.setProperty(INITIAL_CONSISTENCY_CHECK_INTERVAL_MS_PROP, String.valueOf(initialIntevalMs));
return this;
}
public Builder withMaxConsistencyCheckIntervalMs(int maxIntervalMs) {
props.setProperty(MAX_CONSISTENCY_CHECK_INTERVAL_MS_PROP, String.valueOf(maxIntervalMs));
return this;
}
public Builder withMaxConsistencyChecks(int maxConsistencyChecks) {
props.setProperty(MAX_CONSISTENCY_CHECKS_PROP, String.valueOf(maxConsistencyChecks));
return this;
}
public ConsistencyGuardConfig build() {
setDefaultOnCondition(props, !props.containsKey(CONSISTENCY_CHECK_ENABLED_PROP),
CONSISTENCY_CHECK_ENABLED_PROP, DEFAULT_CONSISTENCY_CHECK_ENABLED);
setDefaultOnCondition(props, !props.containsKey(INITIAL_CONSISTENCY_CHECK_INTERVAL_MS_PROP),
INITIAL_CONSISTENCY_CHECK_INTERVAL_MS_PROP, String.valueOf(DEFAULT_INITIAL_CONSISTENCY_CHECK_INTERVAL_MS));
setDefaultOnCondition(props, !props.containsKey(MAX_CONSISTENCY_CHECK_INTERVAL_MS_PROP),
MAX_CONSISTENCY_CHECK_INTERVAL_MS_PROP, String.valueOf(DEFAULT_MAX_CONSISTENCY_CHECK_INTERVAL_MS));
setDefaultOnCondition(props, !props.containsKey(MAX_CONSISTENCY_CHECKS_PROP),
MAX_CONSISTENCY_CHECKS_PROP, String.valueOf(DEFAULT_MAX_CONSISTENCY_CHECKS));
return new ConsistencyGuardConfig(props);
}
}
}

View File

@@ -18,6 +18,7 @@
package com.uber.hoodie.common.util; package com.uber.hoodie.common.util;
import com.google.common.base.Preconditions;
import java.io.FileNotFoundException; import java.io.FileNotFoundException;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
@@ -40,15 +41,12 @@ public class FailSafeConsistencyGuard implements ConsistencyGuard {
private static final transient Logger log = LogManager.getLogger(FailSafeConsistencyGuard.class); private static final transient Logger log = LogManager.getLogger(FailSafeConsistencyGuard.class);
private final FileSystem fs; private final FileSystem fs;
private final int maxAttempts; private final ConsistencyGuardConfig consistencyGuardConfig;
private final long initialDelayMs;
private final long maxDelayMs;
public FailSafeConsistencyGuard(FileSystem fs, int maxAttempts, long initalDelayMs, long maxDelayMs) { public FailSafeConsistencyGuard(FileSystem fs, ConsistencyGuardConfig consistencyGuardConfig) {
this.fs = fs; this.fs = fs;
this.maxAttempts = maxAttempts; this.consistencyGuardConfig = consistencyGuardConfig;
this.initialDelayMs = initalDelayMs; Preconditions.checkArgument(consistencyGuardConfig.isConsistencyCheckEnabled());
this.maxDelayMs = maxDelayMs;
} }
@Override @Override
@@ -121,13 +119,13 @@ public class FailSafeConsistencyGuard implements ConsistencyGuard {
*/ */
private boolean checkFileVisibility(Path filePath, FileVisibility visibility) throws IOException { private boolean checkFileVisibility(Path filePath, FileVisibility visibility) throws IOException {
try { try {
FileStatus[] status = fs.listStatus(filePath); FileStatus status = fs.getFileStatus(filePath);
switch (visibility) { switch (visibility) {
case APPEAR: case APPEAR:
return status.length != 0; return status != null;
case DISAPPEAR: case DISAPPEAR:
default: default:
return status.length == 0; return status == null;
} }
} catch (FileNotFoundException nfe) { } catch (FileNotFoundException nfe) {
switch (visibility) { switch (visibility) {
@@ -147,9 +145,9 @@ public class FailSafeConsistencyGuard implements ConsistencyGuard {
* @throws TimeoutException * @throws TimeoutException
*/ */
private void waitForFileVisibility(Path filePath, FileVisibility visibility) throws TimeoutException { private void waitForFileVisibility(Path filePath, FileVisibility visibility) throws TimeoutException {
long waitMs = initialDelayMs; long waitMs = consistencyGuardConfig.getInitialConsistencyCheckIntervalMs();
int attempt = 0; int attempt = 0;
while (attempt < maxAttempts) { while (attempt < consistencyGuardConfig.getMaxConsistencyChecks()) {
try { try {
if (checkFileVisibility(filePath, visibility)) { if (checkFileVisibility(filePath, visibility)) {
return; return;
@@ -160,7 +158,7 @@ public class FailSafeConsistencyGuard implements ConsistencyGuard {
sleepSafe(waitMs); sleepSafe(waitMs);
waitMs = waitMs * 2; // double check interval every attempt waitMs = waitMs * 2; // double check interval every attempt
waitMs = waitMs > maxDelayMs ? maxDelayMs : waitMs; waitMs = Math.min(waitMs, consistencyGuardConfig.getMaxConsistencyCheckIntervalMs());
attempt++; attempt++;
} }
throw new TimeoutException("Timed-out waiting for the file to " + visibility.name()); throw new TimeoutException("Timed-out waiting for the file to " + visibility.name());
@@ -173,17 +171,17 @@ public class FailSafeConsistencyGuard implements ConsistencyGuard {
* @throws TimeoutException when retries are exhausted * @throws TimeoutException when retries are exhausted
*/ */
private void retryTillSuccess(Function<Integer, Boolean> predicate, String timedOutMessage) throws TimeoutException { private void retryTillSuccess(Function<Integer, Boolean> predicate, String timedOutMessage) throws TimeoutException {
long waitMs = initialDelayMs; long waitMs = consistencyGuardConfig.getInitialConsistencyCheckIntervalMs();
int attempt = 0; int attempt = 0;
log.warn("Max Attempts=" + maxAttempts); log.info("Max Attempts=" + consistencyGuardConfig.getMaxConsistencyChecks());
while (attempt < maxAttempts) { while (attempt < consistencyGuardConfig.getMaxConsistencyChecks()) {
boolean success = predicate.apply(attempt); boolean success = predicate.apply(attempt);
if (success) { if (success) {
return; return;
} }
sleepSafe(waitMs); sleepSafe(waitMs);
waitMs = waitMs * 2; // double check interval every attempt waitMs = waitMs * 2; // double check interval every attempt
waitMs = waitMs > maxDelayMs ? maxDelayMs : waitMs; waitMs = Math.min(waitMs, consistencyGuardConfig.getMaxConsistencyCheckIntervalMs());
attempt++; attempt++;
} }
throw new TimeoutException(timedOutMessage); throw new TimeoutException(timedOutMessage);