diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/CompactionCommand.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/CompactionCommand.java index ffbf70e12a9bc..67445ea873f7e 100644 --- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/CompactionCommand.java +++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/CompactionCommand.java @@ -273,7 +273,7 @@ private String printAllCompactions(HoodieDefaultTimeline timeline, int limit, boolean headerOnly) { - Stream instantsStream = timeline.getCommitsAndCompactionTimeline().getReverseOrderedInstants(); + Stream instantsStream = timeline.getWriteTimeline().getReverseOrderedInstants(); List> compactionPlans = instantsStream .map(instant -> Pair.of(instant, compactionPlanReader.apply(instant))) .filter(pair -> pair.getRight() != null) diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/FileSystemViewCommand.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/FileSystemViewCommand.java index 37bc6517f12d9..d0678fc8579d3 100644 --- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/FileSystemViewCommand.java +++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/FileSystemViewCommand.java @@ -249,7 +249,7 @@ private HoodieTableFileSystemView buildFileSystemView(String globRegex, String m } else if (excludeCompaction) { timeline = metaClient.getActiveTimeline().getCommitsTimeline(); } else { - timeline = metaClient.getActiveTimeline().getCommitsAndCompactionTimeline(); + timeline = metaClient.getActiveTimeline().getWriteTimeline(); } if (!includeInflight) { diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SavepointsCommand.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SavepointsCommand.java index e4d7cc69e9eb1..0ea2fff8d8d19 100644 --- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SavepointsCommand.java +++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SavepointsCommand.java @@ -25,10 +25,12 @@ import org.apache.hudi.cli.utils.SparkUtil; import org.apache.hudi.client.SparkRDDWriteClient; import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.config.HoodieCompactionConfig; import org.apache.hudi.config.HoodieIndexConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieException; @@ -165,8 +167,10 @@ public String deleteSavepoint(@CliOption(key = {"commit"}, help = "Delete a save private static SparkRDDWriteClient createHoodieClient(JavaSparkContext jsc, String basePath) throws Exception { HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath) + .withCompactionConfig(HoodieCompactionConfig.newBuilder() + .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.NEVER).build()) .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()).build(); - return new SparkRDDWriteClient(new HoodieSparkEngineContext(jsc), config, false); + return new SparkRDDWriteClient(new HoodieSparkEngineContext(jsc), config); } } diff --git a/hudi-client/hudi-client-common/pom.xml b/hudi-client/hudi-client-common/pom.xml index 977765f4bd03f..676251e0670de 100644 --- a/hudi-client/hudi-client-common/pom.xml +++ b/hudi-client/hudi-client-common/pom.xml @@ -146,6 +146,14 @@ + + + org.awaitility + awaitility + 3.1.2 + test + + org.junit.jupiter diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieClient.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieClient.java index 0266a65392439..350fe0c9bf7e0 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieClient.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieClient.java @@ -21,6 +21,7 @@ import org.apache.hudi.client.embedded.EmbeddedTimelineServerHelper; import org.apache.hudi.client.embedded.EmbeddedTimelineService; import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.client.heartbeat.HoodieHeartbeatClient; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion; @@ -48,6 +49,7 @@ public abstract class AbstractHoodieClient implements Serializable, AutoCloseabl protected final transient Configuration hadoopConf; protected final HoodieWriteConfig config; protected final String basePath; + protected final HoodieHeartbeatClient heartbeatClient; /** * Timeline Server has the same lifetime as that of Client. Any operations done on the same timeline service will be @@ -70,6 +72,8 @@ protected AbstractHoodieClient(HoodieEngineContext context, HoodieWriteConfig cl this.config = clientConfig; this.timelineServer = timelineServer; shouldStopTimelineServer = !timelineServer.isPresent(); + this.heartbeatClient = new HoodieHeartbeatClient(this.fs, this.basePath, + clientConfig.getHoodieClientHeartbeatIntervalInMs(), clientConfig.getHoodieClientHeartbeatTolerableMisses()); startEmbeddedServerView(); initWrapperFSMetrics(); } @@ -136,4 +140,8 @@ protected HoodieTableMetaClient createMetaClient(boolean loadActiveTimelineOnLoa public Option getTimelineServer() { return timelineServer; } + + public HoodieHeartbeatClient getHeartbeatClient() { + return heartbeatClient; + } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java index 25ce039390ebf..6ce0564e22862 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java @@ -29,8 +29,10 @@ import org.apache.hudi.callback.common.HoodieWriteCommitCallbackMessage; import org.apache.hudi.callback.util.HoodieCommitCallbackFactory; import org.apache.hudi.client.embedded.EmbeddedTimelineService; +import org.apache.hudi.client.heartbeat.HeartbeatUtils; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieCommitMetadata; +import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.HoodieWriteStat; @@ -39,6 +41,7 @@ import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.util.CleanerUtils; import org.apache.hudi.common.util.ClusteringUtils; import org.apache.hudi.common.util.CommitUtils; import org.apache.hudi.common.util.Option; @@ -48,6 +51,7 @@ import org.apache.hudi.config.HoodieCompactionConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieCommitException; +import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.exception.HoodieRestoreException; import org.apache.hudi.exception.HoodieRollbackException; @@ -96,43 +100,29 @@ public abstract class AbstractHoodieWriteClient timelineService) { super(context, writeConfig, timelineService); this.metrics = new HoodieMetrics(config, config.getTableName()); - this.rollbackPending = rollbackPending; this.index = createIndex(writeConfig); } @@ -181,7 +171,7 @@ public boolean commitStats(String instantTime, List stats, Opti HoodieCommitMetadata metadata = CommitUtils.buildMetadata(stats, partitionToReplaceFileIds, extraMetadata, operationType, config.getSchema(), commitActionType); // Finalize write finalizeWrite(table, instantTime, stats); - + HeartbeatUtils.abortIfHeartbeatExpired(instantTime, table, heartbeatClient, config); try { activeTimeline.saveAsComplete(new HoodieInstant(true, commitActionType, instantTime), Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8))); @@ -236,17 +226,16 @@ protected void syncTableMetadata() { * Main API to run bootstrap to hudi. */ public void bootstrap(Option> extraMetadata) { - if (rollbackPending) { - rollBackInflightBootstrap(); - } + // TODO : MULTIWRITER -> check if failed bootstrap files can be cleaned later HoodieTable table = getTableAndInitCtx(WriteOperationType.UPSERT, HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS); + rollbackFailedBootstrap(); table.bootstrap(context, extraMetadata); } /** - * Main API to rollback pending bootstrap. + * Main API to rollback failed bootstrap. */ - protected void rollBackInflightBootstrap() { + public void rollbackFailedBootstrap() { LOG.info("Rolling back pending bootstrap if present"); HoodieTable table = createTable(config, hadoopConf); HoodieTimeline inflightTimeline = table.getMetaClient().getCommitsTimeline().filterPendingExcludingCompaction(); @@ -258,7 +247,6 @@ protected void rollBackInflightBootstrap() { table.rollbackBootstrap(context, HoodieActiveTimeline.createNewInstantTime()); LOG.info("Finished rolling back pending bootstrap"); } - } /** @@ -369,9 +357,8 @@ public abstract O bulkInsertPreppedRecords(I preppedRecords, final String instan /** * Common method containing steps to be performed before write (upsert/insert/... - * - * @param instantTime Instant Time - * @return Write Status + * @param instantTime + * @param writeOperationType */ protected void preWrite(String instantTime, WriteOperationType writeOperationType) { setOperationType(writeOperationType); @@ -424,15 +411,16 @@ protected void postCommit(HoodieTable table, HoodieCommitMetadata me HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(config, table); archiveLog.archiveIfRequired(context); autoCleanOnCommit(); - syncTableMetadata(); } catch (IOException ioe) { throw new HoodieIOException(ioe.getMessage(), ioe); + } finally { + this.heartbeatClient.stop(instantTime); } } protected void runAnyPendingCompactions(HoodieTable table) { - table.getActiveTimeline().getCommitsAndCompactionTimeline().filterPendingCompactionTimeline().getInstants() + table.getActiveTimeline().getWriteTimeline().filterPendingCompactionTimeline().getInstants() .forEach(instant -> { LOG.info("Running previously failed inflight compaction at instant " + instant); compact(instant.getTimestamp(), true); @@ -533,11 +521,14 @@ public void restoreToSavepoint(String savepointTime) { } /** - * Rollback the inflight record changes with the given commit time. + * @Deprecated + * Rollback the inflight record changes with the given commit time. This + * will be removed in future in favor of {@link AbstractHoodieWriteClient#restoreToInstant(String)} * * @param commitInstantTime Instant time of the commit * @throws HoodieRollbackException if rollback cannot be performed successfully */ + @Deprecated public boolean rollback(final String commitInstantTime) throws HoodieRollbackException { LOG.info("Begin rollback of instant " + commitInstantTime); final String rollbackInstantTime = HoodieActiveTimeline.createNewInstantTime(); @@ -598,6 +589,9 @@ public HoodieRestoreMetadata restoreToInstant(final String instantTime) throws H public HoodieCleanMetadata clean(String cleanInstantTime) throws HoodieIOException { LOG.info("Cleaner started"); final Timer.Context timerContext = metrics.getCleanCtx(); + LOG.info("Cleaned failed attempts if any"); + CleanerUtils.rollbackFailedWrites(config.getFailedWritesCleanPolicy(), + HoodieTimeline.CLEAN_ACTION, () -> rollbackFailedWrites()); HoodieCleanMetadata metadata = createTable(config, hadoopConf).clean(context, cleanInstantTime); if (timerContext != null && metadata != null) { long durationMs = metrics.getDurationInMs(timerContext.stop()); @@ -617,11 +611,8 @@ public HoodieCleanMetadata clean() { * Provides a new commit time for a write operation (insert/update/delete). */ public String startCommit() { - // NOTE : Need to ensure that rollback is done before a new commit is started - if (rollbackPending) { - // Only rollback pending commit/delta-commits. Do not touch compaction commits - rollbackPendingCommits(); - } + CleanerUtils.rollbackFailedWrites(config.getFailedWritesCleanPolicy(), + HoodieTimeline.COMMIT_ACTION, () -> rollbackFailedWrites()); String instantTime = HoodieActiveTimeline.createNewInstantTime(); HoodieTableMetaClient metaClient = createMetaClient(true); startCommit(instantTime, metaClient.getCommitActionType(), metaClient); @@ -650,11 +641,8 @@ public void startCommitWithTime(String instantTime, String actionType) { * Completes a new commit time for a write operation (insert/update/delete) with specified action. */ private void startCommitWithTime(String instantTime, String actionType, HoodieTableMetaClient metaClient) { - // NOTE : Need to ensure that rollback is done before a new commit is started - if (rollbackPending) { - // Only rollback inflight commit/delta-commits. Do not touch compaction commits - rollbackPendingCommits(); - } + CleanerUtils.rollbackFailedWrites(config.getFailedWritesCleanPolicy(), + HoodieTimeline.COMMIT_ACTION, () -> rollbackFailedWrites()); startCommit(instantTime, actionType, metaClient); } @@ -666,6 +654,9 @@ private void startCommit(String instantTime, String actionType, HoodieTableMetaC HoodieTimeline.compareTimestamps(latestPending.getTimestamp(), HoodieTimeline.LESSER_THAN, instantTime), "Latest pending compaction instant time must be earlier than this instant time. Latest Compaction :" + latestPending + ", Ingesting at " + instantTime)); + if (config.getFailedWritesCleanPolicy().isLazy()) { + this.heartbeatClient.start(instantTime); + } metaClient.getActiveTimeline().createNewInstant(new HoodieInstant(HoodieInstant.State.REQUESTED, actionType, instantTime)); } @@ -721,7 +712,7 @@ protected abstract void completeCompaction(HoodieCommitMetadata metadata, O writ /** * Rollback failed compactions. Inflight rollbacks for compactions revert the .inflight file to the .requested file - * + * TODO : Deprecate this method and make it protected * @param inflightInstant Inflight Compaction Instant * @param table Hoodie Table */ @@ -749,22 +740,49 @@ private HoodieTimeline getInflightTimelineExcludeCompactionAndClustering(HoodieT } /** - * Cleanup all pending commits. + * Rollback all failed writes. */ - private void rollbackPendingCommits() { + public Boolean rollbackFailedWrites() { HoodieTable table = createTable(config, hadoopConf); - HoodieTimeline inflightTimeline = getInflightTimelineExcludeCompactionAndClustering(table); - List commits = inflightTimeline.getReverseOrderedInstants().map(HoodieInstant::getTimestamp) - .collect(Collectors.toList()); - for (String commit : commits) { - if (HoodieTimeline.compareTimestamps(commit, HoodieTimeline.LESSER_THAN_OR_EQUALS, + List instantsToRollback = getInstantsToRollback(table); + for (String instant : instantsToRollback) { + if (HoodieTimeline.compareTimestamps(instant, HoodieTimeline.LESSER_THAN_OR_EQUALS, HoodieTimeline.FULL_BOOTSTRAP_INSTANT_TS)) { - rollBackInflightBootstrap(); + rollbackFailedBootstrap(); break; } else { - rollback(commit); + rollback(instant); } } + // Delete any heartbeat files for already rolled back commits + try { + HeartbeatUtils.cleanExpiredHeartbeats(this.heartbeatClient.getAllExistingHeartbeatInstants(), + createMetaClient(true), basePath); + } catch (IOException io) { + LOG.error("Unable to delete heartbeat files", io); + } + return true; + } + + private List getInstantsToRollback(HoodieTable table) { + if (config.getFailedWritesCleanPolicy().isEager()) { + HoodieTimeline inflightTimeline = table.getMetaClient().getCommitsTimeline().filterPendingExcludingCompaction(); + return inflightTimeline.getReverseOrderedInstants().map(HoodieInstant::getTimestamp) + .collect(Collectors.toList()); + } else if (config.getFailedWritesCleanPolicy() == HoodieFailedWritesCleaningPolicy.NEVER) { + return Collections.EMPTY_LIST; + } else if (config.getFailedWritesCleanPolicy().isLazy()) { + return table.getMetaClient().getActiveTimeline() + .getCommitsTimeline().filterInflights().getReverseOrderedInstants().filter(instant -> { + try { + return heartbeatClient.isHeartbeatExpired(instant.getTimestamp()); + } catch (IOException io) { + throw new HoodieException("Failed to check heartbeat for instant " + instant, io); + } + }).map(HoodieInstant::getTimestamp).collect(Collectors.toList()); + } else { + throw new IllegalArgumentException("Invalid Failed Writes Cleaning Policy " + config.getFailedWritesCleanPolicy()); + } } /** @@ -912,5 +930,6 @@ public void close() { // Calling this here releases any resources used by your index, so make sure to finish any related operations // before this point this.index.close(); + this.heartbeatClient.stop(); } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/heartbeat/HeartbeatUtils.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/heartbeat/HeartbeatUtils.java new file mode 100644 index 0000000000000..0c69a047fe167 --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/heartbeat/HeartbeatUtils.java @@ -0,0 +1,102 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.client.heartbeat; + +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.table.HoodieTable; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import java.io.File; +import java.io.IOException; +import java.util.List; +import java.util.Set; +import java.util.stream.Collectors; + +/** + * Helper class to delete heartbeat for completed or failed instants with expired heartbeats. + */ +public class HeartbeatUtils { + + private static final Logger LOG = LogManager.getLogger(HeartbeatUtils.class); + + /** + * Deletes the heartbeat file for the specified instant. + * @param fs + * @param basePath + * @param instantTime + * @return + */ + public static boolean deleteHeartbeatFile(FileSystem fs, String basePath, String instantTime) { + boolean deleted = false; + try { + String heartbeatFolderPath = HoodieTableMetaClient.getHeartbeatFolderPath(basePath); + deleted = fs.delete(new Path(heartbeatFolderPath + File.separator + instantTime), false); + if (!deleted) { + LOG.error("Failed to delete heartbeat for instant " + instantTime); + } + } catch (IOException io) { + LOG.error("Unable to delete heartbeat for instant " + instantTime, io); + } + return deleted; + } + + /** + * Deletes the heartbeat files for instants with expired heartbeats without any active instant. + * @param allExistingHeartbeatInstants + * @param metaClient + * @param basePath + */ + public static void cleanExpiredHeartbeats(List allExistingHeartbeatInstants, + HoodieTableMetaClient metaClient, String basePath) { + Set nonExpiredHeartbeatInstants = metaClient.getActiveTimeline() + .filterCompletedInstants().getInstants().map(HoodieInstant::getTimestamp).collect(Collectors.toSet()); + allExistingHeartbeatInstants.stream().forEach(instant -> { + if (!nonExpiredHeartbeatInstants.contains(instant)) { + deleteHeartbeatFile(metaClient.getFs(), basePath, instant); + } + }); + } + + /** + * Check if the heartbeat corresponding to instantTime has expired. If yes, abort by throwing an exception. + * @param instantTime + * @param table + * @param heartbeatClient + * @param config + */ + public static void abortIfHeartbeatExpired(String instantTime, HoodieTable table, + HoodieHeartbeatClient heartbeatClient, HoodieWriteConfig config) { + ValidationUtils.checkArgument(heartbeatClient != null); + try { + if (config.getFailedWritesCleanPolicy().isLazy() && heartbeatClient.isHeartbeatExpired(instantTime)) { + throw new HoodieException("Heartbeat for instant " + instantTime + " has expired, last heartbeat " + + heartbeatClient.getLastHeartbeatTime(table.getMetaClient().getFs(), config.getBasePath(), instantTime)); + } + } catch (IOException io) { + throw new HoodieException("Unable to read heartbeat", io); + } + } +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/heartbeat/HoodieHeartbeatClient.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/heartbeat/HoodieHeartbeatClient.java new file mode 100644 index 0000000000000..044e3441d7578 --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/heartbeat/HoodieHeartbeatClient.java @@ -0,0 +1,289 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.client.heartbeat; + +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.exception.HoodieHeartbeatException; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import javax.annotation.concurrent.NotThreadSafe; +import java.io.File; +import java.io.IOException; +import java.io.OutputStream; +import java.io.Serializable; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import java.util.Timer; +import java.util.TimerTask; + +/** + * This class creates heartbeat for hudi client. This heartbeat is used to ascertain whether the running job is or not. + * NOTE: Due to CPU contention on the driver/client node, the heartbeats could be delayed, hence it's important to set + * the value high enough to avoid that possibility. + */ +@NotThreadSafe +public class HoodieHeartbeatClient implements AutoCloseable, Serializable { + + private static final Logger LOG = LogManager.getLogger(HoodieHeartbeatClient.class); + + private final transient FileSystem fs; + private final String basePath; + // path to the heartbeat folder where all writers are updating their heartbeats + private String heartbeatFolderPath; + // heartbeat interval in millis + private final Long heartbeatIntervalInMs; + private Integer numTolerableHeartbeatMisses; + private final Long maxAllowableHeartbeatIntervalInMs; + private Map instantToHeartbeatMap; + + public HoodieHeartbeatClient(FileSystem fs, String basePath, Long heartbeatIntervalInMs, + Integer numTolerableHeartbeatMisses) { + ValidationUtils.checkArgument(heartbeatIntervalInMs >= 1000, "Cannot set heartbeat lower than 1 second"); + this.fs = fs; + this.basePath = basePath; + this.heartbeatFolderPath = HoodieTableMetaClient.getHeartbeatFolderPath(basePath); + this.heartbeatIntervalInMs = heartbeatIntervalInMs; + this.numTolerableHeartbeatMisses = numTolerableHeartbeatMisses; + this.maxAllowableHeartbeatIntervalInMs = this.heartbeatIntervalInMs * this.numTolerableHeartbeatMisses; + this.instantToHeartbeatMap = new HashMap<>(); + } + + class Heartbeat { + + private String instantTime; + private Boolean isHeartbeatStarted = false; + private Boolean isHeartbeatStopped = false; + private Long lastHeartbeatTime; + private Integer numHeartbeats = 0; + private Timer timer = new Timer(); + + public String getInstantTime() { + return instantTime; + } + + public void setInstantTime(String instantTime) { + this.instantTime = instantTime; + } + + public Boolean isHeartbeatStarted() { + return isHeartbeatStarted; + } + + public void setHeartbeatStarted(Boolean heartbeatStarted) { + isHeartbeatStarted = heartbeatStarted; + } + + public Boolean isHeartbeatStopped() { + return isHeartbeatStopped; + } + + public void setHeartbeatStopped(Boolean heartbeatStopped) { + isHeartbeatStopped = heartbeatStopped; + } + + public Long getLastHeartbeatTime() { + return lastHeartbeatTime; + } + + public void setLastHeartbeatTime(Long lastHeartbeatTime) { + this.lastHeartbeatTime = lastHeartbeatTime; + } + + public Integer getNumHeartbeats() { + return numHeartbeats; + } + + public void setNumHeartbeats(Integer numHeartbeats) { + this.numHeartbeats = numHeartbeats; + } + + public Timer getTimer() { + return timer; + } + + public void setTimer(Timer timer) { + this.timer = timer; + } + + @Override + public String toString() { + return "Heartbeat{" + + "instantTime='" + instantTime + '\'' + + ", isHeartbeatStarted=" + isHeartbeatStarted + + ", isHeartbeatStopped=" + isHeartbeatStopped + + ", lastHeartbeatTime=" + lastHeartbeatTime + + ", numHeartbeats=" + numHeartbeats + + ", timer=" + timer + + '}'; + } + } + + class HeartbeatTask extends TimerTask { + + private final String instantTime; + + HeartbeatTask(String instantTime) { + this.instantTime = instantTime; + } + + @Override + public void run() { + updateHeartbeat(instantTime); + } + } + + /** + * Start a new heartbeat for the specified instant. If there is already one running, this will be a NO_OP + * @param instantTime + */ + public void start(String instantTime) { + LOG.info("Received request to start heartbeat for instant time " + instantTime); + Heartbeat heartbeat = instantToHeartbeatMap.get(instantTime); + ValidationUtils.checkArgument(heartbeat == null || !heartbeat.isHeartbeatStopped(), "Cannot restart a stopped heartbeat for " + instantTime); + if (heartbeat != null && heartbeat.isHeartbeatStarted()) { + // heartbeat already started, NO_OP + return; + } else { + Heartbeat newHeartbeat = new Heartbeat(); + newHeartbeat.setHeartbeatStarted(true); + instantToHeartbeatMap.put(instantTime, newHeartbeat); + // Ensure heartbeat is generated for the first time with this blocking call. + // Since timer submits the task to a thread, no guarantee when that thread will get CPU + // cycles to generate the first heartbeat. + updateHeartbeat(instantTime); + newHeartbeat.getTimer().scheduleAtFixedRate(new HeartbeatTask(instantTime), this.heartbeatIntervalInMs, + this.heartbeatIntervalInMs); + } + } + + /** + * Stops the heartbeat for the specified instant. + * @param instantTime + * @throws HoodieException + */ + public void stop(String instantTime) throws HoodieException { + Heartbeat heartbeat = instantToHeartbeatMap.get(instantTime); + if (heartbeat != null && heartbeat.isHeartbeatStarted() && !heartbeat.isHeartbeatStopped()) { + LOG.info("Stopping heartbeat for instant " + instantTime); + heartbeat.getTimer().cancel(); + heartbeat.setHeartbeatStopped(true); + LOG.info("Stopped heartbeat for instant " + instantTime); + HeartbeatUtils.deleteHeartbeatFile(fs, basePath, instantTime); + LOG.info("Deleted heartbeat file for instant " + instantTime); + } + } + + /** + * Stops all heartbeats started via this instance of the client. + * @throws HoodieException + */ + public void stop() throws HoodieException { + instantToHeartbeatMap.values().stream().forEach(heartbeat -> stop(heartbeat.getInstantTime())); + } + + public static Long getLastHeartbeatTime(FileSystem fs, String basePath, String instantTime) throws IOException { + Path heartbeatFilePath = new Path(HoodieTableMetaClient.getHeartbeatFolderPath(basePath) + File.separator + instantTime); + if (fs.exists(heartbeatFilePath)) { + return fs.getFileStatus(heartbeatFilePath).getModificationTime(); + } else { + // NOTE : This can happen when a writer is upgraded to use lazy cleaning and the last write had failed + return 0L; + } + } + + public static Boolean heartbeatExists(FileSystem fs, String basePath, String instantTime) throws IOException { + Path heartbeatFilePath = new Path(HoodieTableMetaClient.getHeartbeatFolderPath(basePath) + File.separator + instantTime); + if (fs.exists(heartbeatFilePath)) { + return true; + } + return false; + } + + public boolean isHeartbeatExpired(String instantTime) throws IOException { + Long currentTime = System.currentTimeMillis(); + Heartbeat lastHeartbeatForWriter = instantToHeartbeatMap.get(instantTime); + if (lastHeartbeatForWriter == null) { + LOG.info("Heartbeat not found in internal map, falling back to reading from DFS"); + long lastHeartbeatForWriterTime = getLastHeartbeatTime(this.fs, basePath, instantTime); + lastHeartbeatForWriter = new Heartbeat(); + lastHeartbeatForWriter.setLastHeartbeatTime(lastHeartbeatForWriterTime); + lastHeartbeatForWriter.setInstantTime(instantTime); + } + if (currentTime - lastHeartbeatForWriter.getLastHeartbeatTime() > this.maxAllowableHeartbeatIntervalInMs) { + LOG.warn("Heartbeat expired, currentTime = " + currentTime + ", last heartbeat = " + lastHeartbeatForWriter + + ", heartbeat interval = " + this.heartbeatIntervalInMs); + return true; + } + return false; + } + + public List getAllExistingHeartbeatInstants() throws IOException { + Path heartbeatFolder = new Path(heartbeatFolderPath); + if (this.fs.exists(heartbeatFolder)) { + FileStatus[] fileStatus = this.fs.listStatus(new Path(heartbeatFolderPath)); + return Arrays.stream(fileStatus).map(fs -> fs.getPath().getName()).collect(Collectors.toList()); + } + return Collections.EMPTY_LIST; + } + + private void updateHeartbeat(String instantTime) throws HoodieHeartbeatException { + try { + Long newHeartbeatTime = System.currentTimeMillis(); + OutputStream outputStream = + this.fs.create(new Path(heartbeatFolderPath + File.separator + instantTime), true); + outputStream.close(); + Heartbeat heartbeat = instantToHeartbeatMap.get(instantTime); + if (heartbeat.getLastHeartbeatTime() != null && isHeartbeatExpired(instantTime)) { + LOG.error("Aborting, missed generating heartbeat within allowable interval " + this.maxAllowableHeartbeatIntervalInMs); + // Since TimerTask allows only java.lang.Runnable, cannot throw an exception and bubble to the caller thread, hence + // explicitly interrupting the timer thread. + Thread.currentThread().interrupt(); + } + heartbeat.setInstantTime(instantTime); + heartbeat.setLastHeartbeatTime(newHeartbeatTime); + heartbeat.setNumHeartbeats(heartbeat.getNumHeartbeats() + 1); + } catch (IOException io) { + throw new HoodieHeartbeatException("Unable to generate heartbeat ", io); + } + } + + public String getHeartbeatFolderPath() { + return heartbeatFolderPath; + } + + public Heartbeat getHeartbeat(String instantTime) { + return this.instantToHeartbeatMap.get(instantTime); + } + + @Override + public void close() { + this.stop(); + this.instantToHeartbeatMap.clear(); + } +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieCompactionConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieCompactionConfig.java index 934d91a274c26..e05195b9d7767 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieCompactionConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieCompactionConfig.java @@ -20,6 +20,7 @@ import org.apache.hudi.common.config.DefaultHoodieConfig; import org.apache.hudi.common.model.HoodieCleaningPolicy; +import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy; import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload; import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.table.action.compact.CompactionTriggerStrategy; @@ -108,6 +109,9 @@ public class HoodieCompactionConfig extends DefaultHoodieConfig { public static final String COMPACTION_REVERSE_LOG_READ_ENABLED_PROP = "hoodie.compaction.reverse.log.read"; public static final String DEFAULT_COMPACTION_REVERSE_LOG_READ_ENABLED = "false"; private static final String DEFAULT_CLEANER_POLICY = HoodieCleaningPolicy.KEEP_LATEST_COMMITS.name(); + public static final String FAILED_WRITES_CLEANER_POLICY_PROP = "hoodie.failed.writes.cleaner.policy"; + private static final String DEFAULT_FAILED_WRITES_CLEANER_POLICY = + HoodieFailedWritesCleaningPolicy.EAGER.name(); private static final String DEFAULT_AUTO_CLEAN = "true"; private static final String DEFAULT_ASYNC_CLEAN = "false"; private static final String DEFAULT_INLINE_COMPACT = "false"; @@ -276,6 +280,11 @@ public Builder withCleanBootstrapBaseFileEnabled(Boolean cleanBootstrapSourceFil return this; } + public Builder withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy failedWritesPolicy) { + props.setProperty(FAILED_WRITES_CLEANER_POLICY_PROP, failedWritesPolicy.name()); + return this; + } + public HoodieCompactionConfig build() { HoodieCompactionConfig config = new HoodieCompactionConfig(props); setDefaultOnCondition(props, !props.containsKey(AUTO_CLEAN_PROP), AUTO_CLEAN_PROP, DEFAULT_AUTO_CLEAN); @@ -328,6 +337,8 @@ public HoodieCompactionConfig build() { COMMITS_ARCHIVAL_BATCH_SIZE_PROP, DEFAULT_COMMITS_ARCHIVAL_BATCH_SIZE); setDefaultOnCondition(props, !props.containsKey(CLEANER_BOOTSTRAP_BASE_FILE_ENABLED), CLEANER_BOOTSTRAP_BASE_FILE_ENABLED, DEFAULT_CLEANER_BOOTSTRAP_BASE_FILE_ENABLED); + setDefaultOnCondition(props, !props.containsKey(FAILED_WRITES_CLEANER_POLICY_PROP), + FAILED_WRITES_CLEANER_POLICY_PROP, DEFAULT_FAILED_WRITES_CLEANER_POLICY); HoodieCleaningPolicy.valueOf(props.getProperty(CLEANER_POLICY_PROP)); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java index e3c1ef68197f0..24ba10954633d 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java @@ -24,6 +24,7 @@ import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.engine.EngineType; import org.apache.hudi.common.fs.ConsistencyGuardConfig; +import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy; import org.apache.hudi.common.model.HoodieCleaningPolicy; import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload; import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion; @@ -136,6 +137,12 @@ public class HoodieWriteConfig extends DefaultHoodieConfig { private static final String MERGE_ALLOW_DUPLICATE_ON_INSERTS = "hoodie.merge.allow.duplicate.on.inserts"; private static final String DEFAULT_MERGE_ALLOW_DUPLICATE_ON_INSERTS = "false"; + public static final String CLIENT_HEARTBEAT_INTERVAL_IN_MS_PROP = "hoodie.client.heartbeat.interval_in_ms"; + public static final Integer DEFAULT_CLIENT_HEARTBEAT_INTERVAL_IN_MS = 60 * 1000; + + public static final String CLIENT_HEARTBEAT_NUM_TOLERABLE_MISSES_PROP = "hoodie.client.heartbeat.tolerable.misses"; + public static final Integer DEFAULT_CLIENT_HEARTBEAT_NUM_TOLERABLE_MISSES = 2; + /** * HUDI-858 : There are users who had been directly using RDD APIs and have relied on a behavior in 0.4.x to allow * multiple write operations (upsert/buk-insert/...) to be executed within a single commit. @@ -471,6 +478,11 @@ public String getClusteringUpdatesStrategyClass() { return props.getProperty(HoodieClusteringConfig.CLUSTERING_UPDATES_STRATEGY_PROP); } + public HoodieFailedWritesCleaningPolicy getFailedWritesCleanPolicy() { + return HoodieFailedWritesCleaningPolicy + .valueOf(props.getProperty(HoodieCompactionConfig.FAILED_WRITES_CLEANER_POLICY_PROP)); + } + /** * Clustering properties. */ @@ -905,6 +917,18 @@ public Long getMaxMemoryPerPartitionMerge() { return Long.valueOf(props.getProperty(HoodieMemoryConfig.MAX_MEMORY_FOR_MERGE_PROP)); } + public int getMetadataCleanerCommitsRetained() { + return Integer.parseInt(props.getProperty(HoodieMetadataConfig.CLEANER_COMMITS_RETAINED_PROP)); + } + + public Long getHoodieClientHeartbeatIntervalInMs() { + return Long.valueOf(props.getProperty(CLIENT_HEARTBEAT_INTERVAL_IN_MS_PROP)); + } + + public Integer getHoodieClientHeartbeatTolerableMisses() { + return Integer.valueOf(props.getProperty(CLIENT_HEARTBEAT_NUM_TOLERABLE_MISSES_PROP)); + } + /** * File listing metadata configs. */ @@ -936,10 +960,6 @@ public int getMetadataMinCommitsToKeep() { return Integer.parseInt(props.getProperty(HoodieMetadataConfig.MIN_COMMITS_TO_KEEP_PROP)); } - public int getMetadataCleanerCommitsRetained() { - return Integer.parseInt(props.getProperty(HoodieMetadataConfig.CLEANER_COMMITS_RETAINED_PROP)); - } - public static class Builder { protected final Properties props = new Properties(); @@ -1202,6 +1222,16 @@ public Builder withMergeAllowDuplicateOnInserts(boolean routeInsertsToNewFiles) return this; } + public Builder withHeartbeatIntervalInMs(Integer heartbeatIntervalInMs) { + props.setProperty(CLIENT_HEARTBEAT_INTERVAL_IN_MS_PROP, String.valueOf(heartbeatIntervalInMs)); + return this; + } + + public Builder withHeartbeatTolerableMisses(Integer heartbeatTolerableMisses) { + props.setProperty(CLIENT_HEARTBEAT_NUM_TOLERABLE_MISSES_PROP, String.valueOf(heartbeatTolerableMisses)); + return this; + } + public Builder withProperties(Properties properties) { this.props.putAll(properties); return this; @@ -1258,6 +1288,10 @@ protected void setDefaults() { MERGE_DATA_VALIDATION_CHECK_ENABLED, DEFAULT_MERGE_DATA_VALIDATION_CHECK_ENABLED); setDefaultOnCondition(props, !props.containsKey(MERGE_ALLOW_DUPLICATE_ON_INSERTS), MERGE_ALLOW_DUPLICATE_ON_INSERTS, DEFAULT_MERGE_ALLOW_DUPLICATE_ON_INSERTS); + setDefaultOnCondition(props, !props.containsKey(CLIENT_HEARTBEAT_INTERVAL_IN_MS_PROP), + CLIENT_HEARTBEAT_INTERVAL_IN_MS_PROP, String.valueOf(DEFAULT_CLIENT_HEARTBEAT_INTERVAL_IN_MS)); + setDefaultOnCondition(props, !props.containsKey(CLIENT_HEARTBEAT_NUM_TOLERABLE_MISSES_PROP), + CLIENT_HEARTBEAT_NUM_TOLERABLE_MISSES_PROP, String.valueOf(DEFAULT_CLIENT_HEARTBEAT_NUM_TOLERABLE_MISSES)); // Make sure the props is propagated setDefaultOnCondition(props, !isIndexConfigSet, HoodieIndexConfig.newBuilder().withEngineType(engineType).fromProperties(props).build()); @@ -1285,6 +1319,7 @@ protected void setDefaults() { EXTERNAL_RECORD_AND_SCHEMA_TRANSFORMATION, DEFAULT_EXTERNAL_RECORD_AND_SCHEMA_TRANSFORMATION); setDefaultOnCondition(props, !props.containsKey(TIMELINE_LAYOUT_VERSION), TIMELINE_LAYOUT_VERSION, String.valueOf(TimelineLayoutVersion.CURR_VERSION)); + } private void validate() { diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java index be1fc3a72ae6c..003ec7d511595 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java @@ -30,6 +30,7 @@ import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieCleaningPolicy; import org.apache.hudi.common.model.HoodieCommitMetadata; +import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy; import org.apache.hudi.common.model.HoodieFileFormat; import org.apache.hudi.common.model.HoodiePartitionMetadata; import org.apache.hudi.common.model.HoodieRecord; @@ -155,6 +156,7 @@ private HoodieWriteConfig createMetadataWriteConfig(HoodieWriteConfig writeConfi .withAutoClean(false) .withCleanerParallelism(parallelism) .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS) + .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.EAGER) .retainCommits(writeConfig.getMetadataCleanerCommitsRetained()) .archiveCommitsWith(writeConfig.getMetadataMinCommitsToKeep(), writeConfig.getMetadataMaxCommitsToKeep()) // we will trigger compaction manually, to control the instant times diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTimelineArchiveLog.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTimelineArchiveLog.java index 5d4a743c39264..cae5dbbad0107 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTimelineArchiveLog.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTimelineArchiveLog.java @@ -33,6 +33,7 @@ import org.apache.hudi.common.model.HoodieArchivedLogFile; import org.apache.hudi.common.model.HoodieAvroPayload; import org.apache.hudi.common.model.HoodieCommitMetadata; +import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy; import org.apache.hudi.common.model.HoodieReplaceCommitMetadata; import org.apache.hudi.common.model.HoodieRollingStatMetadata; import org.apache.hudi.common.table.HoodieTableMetaClient; @@ -168,13 +169,17 @@ private Stream getCommitInstantsToArchive() { HoodieTimeline commitTimeline = table.getCompletedCommitsTimeline(); Option oldestPendingCompactionInstant = table.getActiveTimeline().filterPendingCompactionTimeline().firstInstant(); + Option oldestInflightCommitInstant = + table.getActiveTimeline() + .getTimelineOfActions(CollectionUtils.createSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION)) + .filterInflights().firstInstant(); // We cannot have any holes in the commit timeline. We cannot archive any commits which are // made after the first savepoint present. Option firstSavepoint = table.getCompletedSavepointTimeline().firstInstant(); if (!commitTimeline.empty() && commitTimeline.countInstants() > maxInstantsToKeep) { // Actually do the commits - return commitTimeline.getInstants() + Stream instantToArchiveStream = commitTimeline.getInstants() .filter(s -> { // if no savepoint present, then dont filter return !(firstSavepoint.isPresent() && HoodieTimeline.compareTimestamps(firstSavepoint.get().getTimestamp(), LESSER_THAN_OR_EQUALS, s.getTimestamp())); @@ -183,7 +188,15 @@ private Stream getCommitInstantsToArchive() { return oldestPendingCompactionInstant .map(instant -> HoodieTimeline.compareTimestamps(instant.getTimestamp(), GREATER_THAN, s.getTimestamp())) .orElse(true); - }).limit(commitTimeline.countInstants() - minInstantsToKeep); + }); + // We need this to ensure that when multiple writers are performing conflict resolution, eligible instants don't + // get archived, i.e, instants after the oldestInflight are retained on the timeline + if (config.getFailedWritesCleanPolicy() == HoodieFailedWritesCleaningPolicy.LAZY) { + instantToArchiveStream = instantToArchiveStream.filter(s -> oldestInflightCommitInstant.map(instant -> + HoodieTimeline.compareTimestamps(instant.getTimestamp(), GREATER_THAN, s.getTimestamp())) + .orElse(true)); + } + return instantToArchiveStream.limit(commitTimeline.countInstants() - minInstantsToKeep); } else { return Stream.empty(); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/BaseScheduleCompactionActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/BaseScheduleCompactionActionExecutor.java index 9e09b8c82761c..37efc0eb0c374 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/BaseScheduleCompactionActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/BaseScheduleCompactionActionExecutor.java @@ -62,7 +62,7 @@ public Option execute() { // Committed and pending compaction instants should have strictly lower timestamps List conflictingInstants = table.getActiveTimeline() - .getCommitsAndCompactionTimeline().getInstants() + .getWriteTimeline().getInstants() .filter(instant -> HoodieTimeline.compareTimestamps( instant.getTimestamp(), HoodieTimeline.GREATER_THAN_OR_EQUALS, instantTime)) .collect(Collectors.toList()); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/restore/BaseRestoreActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/restore/BaseRestoreActionExecutor.java index 0dd2a7cfb4df1..6d38d03eac0d4 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/restore/BaseRestoreActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/restore/BaseRestoreActionExecutor.java @@ -63,7 +63,7 @@ public HoodieRestoreMetadata execute() { restoreTimer.startTimer(); // Get all the commits on the timeline after the provided commit time - List instantsToRollback = table.getActiveTimeline().getCommitsAndCompactionTimeline() + List instantsToRollback = table.getActiveTimeline().getWriteTimeline() .getReverseOrderedInstants() .filter(instant -> HoodieActiveTimeline.GREATER_THAN.test(instant.getTimestamp(), restoreInstantTime)) .collect(Collectors.toList()); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseRollbackActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseRollbackActionExecutor.java index 3a579535528cc..f914302a50df0 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseRollbackActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseRollbackActionExecutor.java @@ -19,6 +19,7 @@ package org.apache.hudi.table.action.rollback; import org.apache.hudi.avro.model.HoodieRollbackMetadata; +import org.apache.hudi.client.heartbeat.HoodieHeartbeatClient; import org.apache.hudi.common.HoodieRollbackStat; import org.apache.hudi.common.bootstrap.index.BootstrapIndex; import org.apache.hudi.common.engine.HoodieEngineContext; @@ -130,23 +131,37 @@ private void validateSavepointRollbacks() { } private void validateRollbackCommitSequence() { - final String instantTimeToRollback = instantToRollback.getTimestamp(); - HoodieTimeline commitTimeline = table.getCompletedCommitsTimeline(); - HoodieTimeline inflightAndRequestedCommitTimeline = table.getPendingCommitTimeline(); - // Make sure only the last n commits are being rolled back - // If there is a commit in-between or after that is not rolled back, then abort - if ((instantTimeToRollback != null) && !commitTimeline.empty() - && !commitTimeline.findInstantsAfter(instantTimeToRollback, Integer.MAX_VALUE).empty()) { - throw new HoodieRollbackException( - "Found commits after time :" + instantTimeToRollback + ", please rollback greater commits first"); - } + // Continue to provide the same behavior if policy is EAGER (similar to pendingRollback logic). This is required + // since with LAZY rollback we support parallel writing which can allow a new inflight while rollback is ongoing + // Remove this once we support LAZY rollback of failed writes by default as parallel writing becomes the default + // writer mode. + if (config.getFailedWritesCleanPolicy().isEager()) { + final String instantTimeToRollback = instantToRollback.getTimestamp(); + HoodieTimeline commitTimeline = table.getCompletedCommitsTimeline(); + HoodieTimeline inflightAndRequestedCommitTimeline = table.getPendingCommitTimeline(); + // Make sure only the last n commits are being rolled back + // If there is a commit in-between or after that is not rolled back, then abort + if ((instantTimeToRollback != null) && !commitTimeline.empty() + && !commitTimeline.findInstantsAfter(instantTimeToRollback, Integer.MAX_VALUE).empty()) { + // check if remnants are from a previous LAZY rollback config, if yes, let out of order rollback continue + try { + if (!HoodieHeartbeatClient.heartbeatExists(table.getMetaClient().getFs(), + config.getBasePath(), instantTimeToRollback)) { + throw new HoodieRollbackException( + "Found commits after time :" + instantTimeToRollback + ", please rollback greater commits first"); + } + } catch (IOException io) { + throw new HoodieRollbackException("Unable to rollback commits ", io); + } + } - List inflights = inflightAndRequestedCommitTimeline.getInstants().map(HoodieInstant::getTimestamp) - .collect(Collectors.toList()); - if ((instantTimeToRollback != null) && !inflights.isEmpty() - && (inflights.indexOf(instantTimeToRollback) != inflights.size() - 1)) { - throw new HoodieRollbackException( - "Found in-flight commits after time :" + instantTimeToRollback + ", please rollback greater commits first"); + List inflights = inflightAndRequestedCommitTimeline.getInstants().map(HoodieInstant::getTimestamp) + .collect(Collectors.toList()); + if ((instantTimeToRollback != null) && !inflights.isEmpty() + && (inflights.indexOf(instantTimeToRollback) != inflights.size() - 1)) { + throw new HoodieRollbackException( + "Found in-flight commits after time :" + instantTimeToRollback + ", please rollback greater commits first"); + } } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/savepoint/SavepointHelpers.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/savepoint/SavepointHelpers.java index 0d51a639aa03a..c12a4f169b9d0 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/savepoint/SavepointHelpers.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/savepoint/SavepointHelpers.java @@ -52,7 +52,7 @@ public static void validateSavepointRestore(HoodieTable table, String savepointT // Make sure the restore was successful table.getMetaClient().reloadActiveTimeline(); Option lastInstant = table.getActiveTimeline() - .getCommitsAndCompactionTimeline() + .getWriteTimeline() .filterCompletedAndCompactionInstants() .lastInstant(); ValidationUtils.checkArgument(lastInstant.isPresent()); diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/heartbeat/TestHoodieHeartbeatClient.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/heartbeat/TestHoodieHeartbeatClient.java new file mode 100644 index 0000000000000..88fe28edb4e7c --- /dev/null +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/heartbeat/TestHoodieHeartbeatClient.java @@ -0,0 +1,93 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.client.heartbeat; + +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hudi.common.testutils.HoodieCommonTestHarness; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.io.IOException; + +import static java.util.concurrent.TimeUnit.SECONDS; +import static org.awaitility.Awaitility.await; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; + +public class TestHoodieHeartbeatClient extends HoodieCommonTestHarness { + + private static String instantTime1 = "100"; + private static String instantTime2 = "101"; + private static Long heartBeatInterval = 1000L; + private static int numTolerableMisses = 1; + + @BeforeEach + public void init() throws IOException { + initMetaClient(); + } + + @Test + public void testStartHeartbeat() throws IOException { + HoodieHeartbeatClient hoodieHeartbeatClient = + new HoodieHeartbeatClient(metaClient.getFs(), metaClient.getBasePath(), heartBeatInterval, numTolerableMisses); + hoodieHeartbeatClient.start(instantTime1); + FileStatus [] fs = metaClient.getFs().listStatus(new Path(hoodieHeartbeatClient.getHeartbeatFolderPath())); + assertTrue(fs.length == 1); + assertTrue(fs[0].getPath().toString().contains(instantTime1)); + } + + @Test + public void testStopHeartbeat() { + HoodieHeartbeatClient hoodieHeartbeatClient = + new HoodieHeartbeatClient(metaClient.getFs(), metaClient.getBasePath(), heartBeatInterval, numTolerableMisses); + hoodieHeartbeatClient.start(instantTime1); + hoodieHeartbeatClient.stop(instantTime1); + await().atMost(5, SECONDS).until(() -> hoodieHeartbeatClient.getHeartbeat(instantTime1).getNumHeartbeats() > 0); + Integer numHeartBeats = hoodieHeartbeatClient.getHeartbeat(instantTime1).getNumHeartbeats(); + assertTrue(numHeartBeats == 1); + } + + @Test + public void testIsHeartbeatExpired() throws IOException { + HoodieHeartbeatClient hoodieHeartbeatClient = + new HoodieHeartbeatClient(metaClient.getFs(), metaClient.getBasePath(), heartBeatInterval, numTolerableMisses); + hoodieHeartbeatClient.start(instantTime1); + hoodieHeartbeatClient.stop(instantTime1); + assertFalse(hoodieHeartbeatClient.isHeartbeatExpired(instantTime1)); + } + + @Test + public void testNumHeartbeatsGenerated() { + Long heartBeatInterval = 5000L; + HoodieHeartbeatClient hoodieHeartbeatClient = + new HoodieHeartbeatClient(metaClient.getFs(), metaClient.getBasePath(), heartBeatInterval, numTolerableMisses); + hoodieHeartbeatClient.start("100"); + await().atMost(5, SECONDS).until(() -> hoodieHeartbeatClient.getHeartbeat(instantTime1).getNumHeartbeats() >= 1); + } + + @Test + public void testDeleteWrongHeartbeat() throws IOException { + HoodieHeartbeatClient hoodieHeartbeatClient = + new HoodieHeartbeatClient(metaClient.getFs(), metaClient.getBasePath(), heartBeatInterval, numTolerableMisses); + hoodieHeartbeatClient.start(instantTime1); + hoodieHeartbeatClient.stop(instantTime1); + assertFalse(HeartbeatUtils.deleteHeartbeatFile(metaClient.getFs(), basePath, instantTime2)); + } +} diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/HoodieFlinkWriteClient.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/HoodieFlinkWriteClient.java index 9a7f5e8e4acbf..0f1557f0339dd 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/HoodieFlinkWriteClient.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/HoodieFlinkWriteClient.java @@ -74,14 +74,16 @@ public HoodieFlinkWriteClient(HoodieEngineContext context, HoodieWriteConfig cli this(context, clientConfig, false); } + @Deprecated public HoodieFlinkWriteClient(HoodieEngineContext context, HoodieWriteConfig writeConfig, boolean rollbackPending) { - super(context, writeConfig, rollbackPending); + super(context, writeConfig); this.bucketToHandles = new HashMap<>(); } + @Deprecated public HoodieFlinkWriteClient(HoodieEngineContext context, HoodieWriteConfig writeConfig, boolean rollbackPending, Option timelineService) { - super(context, writeConfig, rollbackPending, timelineService); + super(context, writeConfig, timelineService); } /** diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/HoodieJavaWriteClient.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/HoodieJavaWriteClient.java index a162663d906d5..f252e8b94d743 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/HoodieJavaWriteClient.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/HoodieJavaWriteClient.java @@ -57,7 +57,7 @@ public HoodieJavaWriteClient(HoodieEngineContext context, HoodieWriteConfig writeConfig, boolean rollbackPending, Option timelineService) { - super(context, writeConfig, rollbackPending, timelineService); + super(context, writeConfig, timelineService); } @Override diff --git a/hudi-client/hudi-spark-client/pom.xml b/hudi-client/hudi-spark-client/pom.xml index 20f0565976468..045b8e781c507 100644 --- a/hudi-client/hudi-spark-client/pom.xml +++ b/hudi-client/hudi-spark-client/pom.xml @@ -167,6 +167,12 @@ junit-platform-commons test + + + org.awaitility + awaitility + test + diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkRDDWriteClient.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkRDDWriteClient.java index 6abefbd276f48..091f1dafa8051 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkRDDWriteClient.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkRDDWriteClient.java @@ -75,13 +75,20 @@ public SparkRDDWriteClient(HoodieEngineContext context, HoodieWriteConfig client super(context, clientConfig); } + @Deprecated public SparkRDDWriteClient(HoodieEngineContext context, HoodieWriteConfig writeConfig, boolean rollbackPending) { - super(context, writeConfig, rollbackPending); + super(context, writeConfig); } + @Deprecated public SparkRDDWriteClient(HoodieEngineContext context, HoodieWriteConfig writeConfig, boolean rollbackPending, Option timelineService) { - super(context, writeConfig, rollbackPending, timelineService); + super(context, writeConfig, timelineService); + } + + public SparkRDDWriteClient(HoodieEngineContext context, HoodieWriteConfig writeConfig, + Option timelineService) { + super(context, writeConfig, timelineService); } /** @@ -131,9 +138,6 @@ public JavaRDD> filterExists(JavaRDD> hoodieReco */ @Override public void bootstrap(Option> extraMetadata) { - if (rollbackPending) { - rollBackInflightBootstrap(); - } getTableAndInitCtx(WriteOperationType.UPSERT, HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS).bootstrap(context, extraMetadata); } diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestClientRollback.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestClientRollback.java index 0599ee8d82b79..b4b5c05dfef51 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestClientRollback.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestClientRollback.java @@ -21,6 +21,7 @@ import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieCleaningPolicy; +import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieInstant; @@ -200,14 +201,11 @@ public void testRollbackCommit() throws Exception { .withBaseFilesInPartitions(partitionAndFileId3); HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath) + .withCompactionConfig(HoodieCompactionConfig.newBuilder() + .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY).build()) .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.INMEMORY).build()).build(); - try (SparkRDDWriteClient client = getHoodieWriteClient(config, false)) { - - // Rollback commit 1 (this should fail, since commit2 is still around) - assertThrows(HoodieRollbackException.class, () -> { - client.rollback(commitTime1); - }, "Should have thrown an exception "); + try (SparkRDDWriteClient client = getHoodieWriteClient(config)) { // Rollback commit3 client.rollback(commitTime3); @@ -290,12 +288,14 @@ public void testAutoRollbackInflightCommit() throws Exception { .addInflightCommit(commitTime3) .withBaseFilesInPartitions(partitionAndFileId3); - // Turn auto rollback off + // Set Failed Writes rollback to LAZY HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath) - .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.INMEMORY).build()).build(); + .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.INMEMORY).build()) + .withCompactionConfig(HoodieCompactionConfig.newBuilder() + .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY).build()).build(); final String commitTime4 = "20160506030621"; - try (SparkRDDWriteClient client = getHoodieWriteClient(config, false)) { + try (SparkRDDWriteClient client = getHoodieWriteClient(config)) { client.startCommitWithTime(commitTime4); // Check results, nothing changed assertTrue(testTable.commitExists(commitTime1)); @@ -306,9 +306,11 @@ public void testAutoRollbackInflightCommit() throws Exception { assertTrue(testTable.baseFilesExist(partitionAndFileId3, commitTime3)); } - // Turn auto rollback on + // Set Failed Writes rollback to EAGER + config = HoodieWriteConfig.newBuilder().withPath(basePath) + .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.INMEMORY).build()).build(); final String commitTime5 = "20160506030631"; - try (SparkRDDWriteClient client = getHoodieWriteClient(config, true)) { + try (SparkRDDWriteClient client = getHoodieWriteClient(config)) { client.startCommitWithTime(commitTime5); assertTrue(testTable.commitExists(commitTime1)); assertFalse(testTable.inflightCommitExists(commitTime2)); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestHoodieClientOnCopyOnWriteStorage.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestHoodieClientOnCopyOnWriteStorage.java index c81aa114aa6f0..9383854301c8a 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestHoodieClientOnCopyOnWriteStorage.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestHoodieClientOnCopyOnWriteStorage.java @@ -23,10 +23,12 @@ import org.apache.hadoop.fs.Path; import org.apache.hudi.avro.model.HoodieClusteringPlan; import org.apache.hudi.avro.model.HoodieRequestedReplaceMetadata; +import org.apache.hudi.avro.model.HoodieCleanMetadata; import org.apache.hudi.common.fs.ConsistencyGuardConfig; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieCommitMetadata; +import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy; import org.apache.hudi.common.model.HoodieFileGroupId; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; @@ -43,8 +45,10 @@ import org.apache.hudi.common.table.view.TableFileSystemView.BaseFileOnlyView; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.common.testutils.HoodieTestTable; +import org.apache.hudi.common.testutils.HoodieTestUtils; import org.apache.hudi.common.testutils.RawTripTestPayload; import org.apache.hudi.common.util.ClusteringUtils; +import org.apache.hudi.common.util.CollectionUtils; import org.apache.hudi.common.util.FileIOUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.ParquetUtils; @@ -95,6 +99,9 @@ import java.util.Map; import java.util.Set; import java.util.UUID; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; import java.util.stream.Collectors; import java.util.stream.Stream; import java.util.Properties; @@ -283,7 +290,8 @@ private void testDeduplication( // Perform write-action and check JavaRDD recordList = jsc.parallelize(Arrays.asList(recordOne, recordTwo, recordThree), 1); - try (SparkRDDWriteClient client = getHoodieWriteClient(getConfigBuilder().combineInput(true, true).build(), false);) { + try (SparkRDDWriteClient client = getHoodieWriteClient(getConfigBuilder(HoodieFailedWritesCleaningPolicy.LAZY) + .combineInput(true, true).build());) { client.startCommitWithTime(newCommitTime); List statuses = writeFn.apply(client, recordList, newCommitTime).collect(); assertNoWriteErrors(statuses); @@ -338,12 +346,13 @@ private void testUpsertsInternal(HoodieWriteConfig config, Function3, SparkRDDWriteClient, JavaRDD, String> writeFn, boolean isPrepped) throws Exception { // Force using older timeline layout - HoodieWriteConfig hoodieWriteConfig = getConfigBuilder().withProps(config.getProps()).withTimelineLayoutVersion( + HoodieWriteConfig hoodieWriteConfig = getConfigBuilder(HoodieFailedWritesCleaningPolicy.LAZY) + .withProps(config.getProps()).withTimelineLayoutVersion( VERSION_0).build(); HoodieTableMetaClient.initTableType(metaClient.getHadoopConf(), metaClient.getBasePath(), metaClient.getTableType(), metaClient.getTableConfig().getTableName(), metaClient.getArchivePath(), metaClient.getTableConfig().getPayloadClass(), VERSION_0); - SparkRDDWriteClient client = getHoodieWriteClient(hoodieWriteConfig, false); + SparkRDDWriteClient client = getHoodieWriteClient(hoodieWriteConfig); // Write 1 (only inserts) String newCommitTime = "001"; @@ -373,7 +382,7 @@ private void testUpsertsInternal(HoodieWriteConfig config, // Now simulate an upgrade and perform a restore operation HoodieWriteConfig newConfig = getConfigBuilder().withProps(config.getProps()).withTimelineLayoutVersion( TimelineLayoutVersion.CURR_VERSION).build(); - client = getHoodieWriteClient(newConfig, false); + client = getHoodieWriteClient(newConfig); client.restoreToInstant("004"); // Check the entire dataset has all records still @@ -488,7 +497,7 @@ private void testHoodieConcatHandle(HoodieWriteConfig config, boolean isPrepped) HoodieTableMetaClient.initTableType(metaClient.getHadoopConf(), metaClient.getBasePath(), metaClient.getTableType(), metaClient.getTableConfig().getTableName(), metaClient.getArchivePath(), metaClient.getTableConfig().getPayloadClass(), VERSION_0); - SparkRDDWriteClient client = getHoodieWriteClient(hoodieWriteConfig, false); + SparkRDDWriteClient client = getHoodieWriteClient(hoodieWriteConfig); // Write 1 (only inserts) String newCommitTime = "001"; @@ -508,7 +517,7 @@ private void testHoodieConcatHandle(HoodieWriteConfig config, boolean isPrepped) writeBatch(client, newCommitTime, prevCommitTime, Option.of(Arrays.asList(commitTimeBetweenPrevAndNew)), initCommitTime, numRecords, recordGenFunction, SparkRDDWriteClient::insert, true, numRecords, 300, - 2); + 2, false); } /** @@ -516,8 +525,7 @@ private void testHoodieConcatHandle(HoodieWriteConfig config, boolean isPrepped) */ @Test public void testDeletes() throws Exception { - SparkRDDWriteClient client = getHoodieWriteClient(getConfig(), false); - + SparkRDDWriteClient client = getHoodieWriteClient(getConfigBuilder(HoodieFailedWritesCleaningPolicy.LAZY).build()); /** * Write 1 (inserts and deletes) Write actual 200 insert records and ignore 100 delete records */ @@ -536,7 +544,7 @@ public void testDeletes() throws Exception { }; writeBatch(client, newCommitTime, initCommitTime, Option.empty(), initCommitTime, // unused as genFn uses hard-coded number of inserts/updates/deletes - -1, recordGenFunction, SparkRDDWriteClient::upsert, true, 200, 200, 1); + -1, recordGenFunction, SparkRDDWriteClient::upsert, true, 200, 200, 1, false); /** * Write 2 (deletes+writes). @@ -553,7 +561,7 @@ public void testDeletes() throws Exception { return recordsInSecondBatch; }; writeBatch(client, newCommitTime, prevCommitTime, Option.empty(), initCommitTime, 100, recordGenFunction, - SparkRDDWriteClient::upsert, true, 50, 150, 2); + SparkRDDWriteClient::upsert, true, 50, 150, 2, false); } /** @@ -563,8 +571,7 @@ public void testDeletes() throws Exception { */ @Test public void testDeletesForInsertsInSameBatch() throws Exception { - SparkRDDWriteClient client = getHoodieWriteClient(getConfig(), false); - + SparkRDDWriteClient client = getHoodieWriteClient(getConfigBuilder(HoodieFailedWritesCleaningPolicy.LAZY).build()); /** * Write 200 inserts and issue deletes to a subset(50) of inserts. */ @@ -583,7 +590,7 @@ public void testDeletesForInsertsInSameBatch() throws Exception { }; writeBatch(client, newCommitTime, initCommitTime, Option.empty(), initCommitTime, - -1, recordGenFunction, SparkRDDWriteClient::upsert, true, 150, 150, 1); + -1, recordGenFunction, SparkRDDWriteClient::upsert, true, 150, 150, 1, false); } /** @@ -625,7 +632,10 @@ private void testUpsertsUpdatePartitionPath(IndexType indexType, HoodieWriteConf HoodieTableMetaClient.initTableType(metaClient.getHadoopConf(), metaClient.getBasePath(), metaClient.getTableType(), metaClient.getTableConfig().getTableName(), metaClient.getArchivePath(), metaClient.getTableConfig().getPayloadClass(), VERSION_0); - SparkRDDWriteClient client = getHoodieWriteClient(hoodieWriteConfig, false); + // Set rollback to LAZY so no inflights are deleted + hoodieWriteConfig.getProps().put(HoodieCompactionConfig.FAILED_WRITES_CLEANER_POLICY_PROP, + HoodieFailedWritesCleaningPolicy.LAZY.name()); + SparkRDDWriteClient client = getHoodieWriteClient(hoodieWriteConfig); // Write 1 String newCommitTime = "001"; @@ -779,7 +789,7 @@ public void testUpdateRejectForClustering() throws IOException { props.setProperty(ASYNC_CLUSTERING_ENABLE_OPT_KEY, "true"); HoodieWriteConfig config = getSmallInsertWriteConfig(100, TRIP_EXAMPLE_SCHEMA, dataGen.getEstimatedFileSizeInBytes(150), props); - SparkRDDWriteClient client = getHoodieWriteClient(config, false); + SparkRDDWriteClient client = getHoodieWriteClient(config); HoodieSparkCopyOnWriteTable table = (HoodieSparkCopyOnWriteTable) HoodieSparkTable.create(config, context, metaClient); //1. insert to generate 2 file group @@ -834,8 +844,7 @@ public void testSmallInsertHandlingForUpserts() throws Exception { // setup the small file handling params HoodieWriteConfig config = getSmallInsertWriteConfig(insertSplitLimit); // hold upto 200 records max dataGen = new HoodieTestDataGenerator(new String[] {testPartitionPath}); - - SparkRDDWriteClient client = getHoodieWriteClient(config, false); + SparkRDDWriteClient client = getHoodieWriteClient(config); // Inserts => will write file1 String commitTime1 = "001"; @@ -946,7 +955,7 @@ public void testSmallInsertHandlingForInserts(boolean mergeAllowDuplicateInserts // setup the small file handling params HoodieWriteConfig config = getSmallInsertWriteConfig(insertSplitLimit, false, mergeAllowDuplicateInserts); // hold upto 200 records max dataGen = new HoodieTestDataGenerator(new String[] {testPartitionPath}); - SparkRDDWriteClient client = getHoodieWriteClient(config, false); + SparkRDDWriteClient client = getHoodieWriteClient(config); // Inserts => will write file1 String commitTime1 = "001"; @@ -1026,7 +1035,7 @@ public void testDeletesWithDeleteApi() throws Exception { HoodieWriteConfig config = getSmallInsertWriteConfig(insertSplitLimit); // hold upto 200 records max dataGen = new HoodieTestDataGenerator(new String[] {testPartitionPath}); - SparkRDDWriteClient client = getHoodieWriteClient(config, false); + SparkRDDWriteClient client = getHoodieWriteClient(config); // Inserts => will write file1 String commitTime1 = "001"; @@ -1104,7 +1113,7 @@ public void testClusteringWithSortColumns() throws Exception { private void testClustering(HoodieClusteringConfig clusteringConfig) throws Exception { // create config to not update small files. HoodieWriteConfig config = getSmallInsertWriteConfig(2000, false, 10); - SparkRDDWriteClient client = getHoodieWriteClient(config, false); + SparkRDDWriteClient client = getHoodieWriteClient(config); dataGen = new HoodieTestDataGenerator(); String commitTime = "100"; List records1 = dataGen.generateInserts(commitTime, 200); @@ -1120,10 +1129,10 @@ private void testClustering(HoodieClusteringConfig clusteringConfig) throws Exce fileIdIntersection.retainAll(fileIds2); assertEquals(0, fileIdIntersection.size()); - config = getConfigBuilder().withClusteringConfig(clusteringConfig).build(); + config = getConfigBuilder(HoodieFailedWritesCleaningPolicy.LAZY).withClusteringConfig(clusteringConfig).build(); // create client with new config. - client = getHoodieWriteClient(config, false); + client = getHoodieWriteClient(config); String clusteringCommitTime = client.scheduleClustering(Option.empty()).get().toString(); HoodieWriteMetadata> clusterMetadata = client.cluster(clusteringCommitTime, true); List allRecords = Stream.concat(records1.stream(), records2.stream()).collect(Collectors.toList()); @@ -1176,7 +1185,7 @@ public void testInsertOverwritePartitionHandlingWithSimilarNumberOfRecords() thr private void verifyInsertOverwritePartitionHandling(int batch1RecordsCount, int batch2RecordsCount) throws Exception { final String testPartitionPath = "americas"; HoodieWriteConfig config = getSmallInsertWriteConfig(2000, false); - SparkRDDWriteClient client = getHoodieWriteClient(config, false); + SparkRDDWriteClient client = getHoodieWriteClient(config); dataGen = new HoodieTestDataGenerator(new String[] {testPartitionPath}); // Do Inserts @@ -1257,7 +1266,7 @@ private Set deletePartitionWithCommit(SparkRDDWriteClient client, String */ private void verifyDeletePartitionsHandling(int batch1RecordsCount, int batch2RecordsCount, int batch3RecordsCount) throws Exception { HoodieWriteConfig config = getSmallInsertWriteConfig(2000, false); - SparkRDDWriteClient client = getHoodieWriteClient(config, false); + SparkRDDWriteClient client = getHoodieWriteClient(config); dataGen = new HoodieTestDataGenerator(); // Do Inserts for DEFAULT_FIRST_PARTITION_PATH @@ -1405,8 +1414,7 @@ public void testDeletesWithoutInserts() { // setup the small file handling params HoodieWriteConfig config = getSmallInsertWriteConfig(insertSplitLimit, true); // hold upto 200 records max dataGen = new HoodieTestDataGenerator(new String[] {testPartitionPath}); - - SparkRDDWriteClient client = getHoodieWriteClient(config, false); + SparkRDDWriteClient client = getHoodieWriteClient(config); // delete non existent keys String commitTime1 = "001"; @@ -1612,6 +1620,181 @@ public void testRollbackAfterConsistencyCheckFailureUsingMarkers(boolean enableO testRollbackAfterConsistencyCheckFailureUsingFileList(true, enableOptimisticConsistencyGuard); } + @ParameterizedTest + @EnumSource(value = HoodieFailedWritesCleaningPolicy.class, names = {"LAZY", "NEVER"}) + public void testRollbackFailedCommits(HoodieFailedWritesCleaningPolicy cleaningPolicy) throws Exception { + HoodieTestUtils.init(hadoopConf, basePath); + // Perform 2 failed writes to table + SparkRDDWriteClient client = new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy)); + writeBatch(client, "100", "100", Option.of(Arrays.asList("100")), "100", + 100, dataGen::generateInserts, SparkRDDWriteClient::bulkInsert, false, 100, 300, + 0, false); + client.close(); + client = new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy)); + writeBatch(client, "200", "100", Option.of(Arrays.asList("200")), "200", + 100, dataGen::generateInserts, SparkRDDWriteClient::bulkInsert, false, 100, 300, + 0, false); + client.close(); + // refresh data generator to delete records generated from failed commits + dataGen = new HoodieTestDataGenerator(); + // Perform 1 successful write + client = new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy)); + writeBatch(client, "300", "200", Option.of(Arrays.asList("300")), "300", + 100, dataGen::generateInserts, SparkRDDWriteClient::bulkInsert, false, 100, 300, + 0, true); + HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setConf(hadoopConf).setBasePath(basePath).build(); + + assertTrue(metaClient.getActiveTimeline().getTimelineOfActions( + CollectionUtils.createSet(HoodieTimeline.ROLLBACK_ACTION)).countInstants() == 0); + assertTrue(metaClient.getActiveTimeline().filterInflights().countInstants() == 2); + assertTrue(metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants().countInstants() == 1); + // Await till enough time passes such that the first 2 failed commits heartbeats are expired + boolean conditionMet = false; + while (!conditionMet) { + conditionMet = client.getHeartbeatClient().isHeartbeatExpired("200"); + Thread.sleep(2000); + } + client = new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy)); + // Perform 1 successful write + writeBatch(client, "400", "300", Option.of(Arrays.asList("400")), "400", + 100, dataGen::generateInserts, SparkRDDWriteClient::bulkInsert, false, 100, 300, + 0, true); + client.clean(); + HoodieActiveTimeline timeline = metaClient.getActiveTimeline().reload(); + if (cleaningPolicy.isLazy()) { + assertTrue( + timeline + .getTimelineOfActions(CollectionUtils.createSet(HoodieTimeline.ROLLBACK_ACTION)) + .countInstants() + == 2); + // Since we write rollbacks not clean, there should be no clean action on the timeline + assertTrue( + timeline + .getTimelineOfActions(CollectionUtils.createSet(HoodieTimeline.CLEAN_ACTION)) + .countInstants() + == 0); + assertTrue(timeline.getCommitsTimeline().filterCompletedInstants().countInstants() == 2); + } else if (cleaningPolicy.isNever()) { + assertTrue( + timeline + .getTimelineOfActions(CollectionUtils.createSet(HoodieTimeline.ROLLBACK_ACTION)) + .countInstants() + == 0); + // There should be no clean or rollback action on the timeline + assertTrue( + timeline + .getTimelineOfActions(CollectionUtils.createSet(HoodieTimeline.CLEAN_ACTION)) + .countInstants() + == 0); + assertTrue(timeline.getCommitsTimeline().filterCompletedInstants().countInstants() == 2); + } + } + + @Test + public void testRollbackFailedCommitsToggleCleaningPolicy() throws Exception { + HoodieTestUtils.init(hadoopConf, basePath); + HoodieFailedWritesCleaningPolicy cleaningPolicy = HoodieFailedWritesCleaningPolicy.EAGER; + SparkRDDWriteClient client = new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy)); + // Perform 1 failed writes to table + writeBatch(client, "100", "100", Option.of(Arrays.asList("100")), "100", + 100, dataGen::generateInserts, SparkRDDWriteClient::bulkInsert, false, 100, 300, + 0, false); + client.close(); + // Toggle cleaning policy to LAZY + cleaningPolicy = HoodieFailedWritesCleaningPolicy.LAZY; + // Perform 2 failed writes to table + client = new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy)); + writeBatch(client, "200", "100", Option.of(Arrays.asList("200")), "200", + 100, dataGen::generateInserts, SparkRDDWriteClient::bulkInsert, false, 100, 300, + 0, false); + client.close(); + client = new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy)); + writeBatch(client, "300", "200", Option.of(Arrays.asList("300")), "300", + 100, dataGen::generateInserts, SparkRDDWriteClient::bulkInsert, false, 100, 300, + 0, false); + client.close(); + // Await till enough time passes such that the first 2 failed commits heartbeats are expired + boolean conditionMet = false; + while (!conditionMet) { + conditionMet = client.getHeartbeatClient().isHeartbeatExpired("300"); + Thread.sleep(2000); + } + client.clean(); + HoodieActiveTimeline timeline = metaClient.getActiveTimeline().reload(); + assertTrue(timeline.getTimelineOfActions( + CollectionUtils.createSet(HoodieTimeline.ROLLBACK_ACTION)).countInstants() == 3); + // Perform 2 failed commits + client = new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy)); + writeBatch(client, "400", "300", Option.of(Arrays.asList("400")), "400", + 100, dataGen::generateInserts, SparkRDDWriteClient::bulkInsert, false, 100, 300, + 0, false); + client.close(); + client = new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy)); + writeBatch(client, "500", "400", Option.of(Arrays.asList("500")), "500", + 100, dataGen::generateInserts, SparkRDDWriteClient::bulkInsert, false, 100, 300, + 0, false); + client.close(); + // Toggle cleaning policy to EAGER + cleaningPolicy = HoodieFailedWritesCleaningPolicy.EAGER; + client = new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy)); + client.startCommit(); + timeline = metaClient.getActiveTimeline().reload(); + assertTrue(timeline.getTimelineOfActions( + CollectionUtils.createSet(HoodieTimeline.ROLLBACK_ACTION)).countInstants() == 5); + assertTrue(timeline.getCommitsTimeline().filterCompletedInstants().countInstants() == 0); + } + + @Test + public void testParallelInsertAndCleanPreviousFailedCommits() throws Exception { + HoodieFailedWritesCleaningPolicy cleaningPolicy = HoodieFailedWritesCleaningPolicy.LAZY; + ExecutorService service = Executors.newFixedThreadPool(2); + HoodieTestUtils.init(hadoopConf, basePath); + // Perform 2 failed writes to table + SparkRDDWriteClient client = new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy)); + writeBatch(client, "100", "100", Option.of(Arrays.asList("100")), "100", + 100, dataGen::generateInserts, SparkRDDWriteClient::bulkInsert, false, 100, 100, + 0, false); + client.close(); + client = new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy)); + writeBatch(client, "200", "200", Option.of(Arrays.asList("200")), "200", + 100, dataGen::generateInserts, SparkRDDWriteClient::bulkInsert, false, 100, 100, + 0, false); + client.close(); + // refresh data generator to delete records generated from failed commits + dataGen = new HoodieTestDataGenerator(); + // Create a succesful commit + Future> commit3 = service.submit(() -> writeBatch(new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy)), + "300", "200", Option.of(Arrays.asList("300")), "200", 100, dataGen::generateInserts, + SparkRDDWriteClient::bulkInsert, false, 100, 100, 0, true)); + commit3.get(); + HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setConf(hadoopConf).setBasePath(basePath).build(); + + assertTrue(metaClient.getActiveTimeline().getTimelineOfActions( + CollectionUtils.createSet(HoodieTimeline.ROLLBACK_ACTION)).countInstants() == 0); + assertTrue(metaClient.getActiveTimeline().filterInflights().countInstants() == 2); + assertTrue(metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants().countInstants() == 1); + client = new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy)); + // Await till enough time passes such that the first 2 failed commits heartbeats are expired + boolean conditionMet = false; + while (!conditionMet) { + conditionMet = client.getHeartbeatClient().isHeartbeatExpired("200"); + Thread.sleep(2000); + } + Future> commit4 = service.submit(() -> writeBatch(new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy)), + "400", "300", Option.of(Arrays.asList("400")), "400", 100, dataGen::generateInserts, + SparkRDDWriteClient::bulkInsert, false, 100, 100, 0, true)); + Future clean1 = service.submit(() -> new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy)).clean()); + commit4.get(); + clean1.get(); + HoodieActiveTimeline timeline = metaClient.getActiveTimeline().reload(); + assertTrue(timeline.getTimelineOfActions( + CollectionUtils.createSet(HoodieTimeline.ROLLBACK_ACTION)).countInstants() == 2); + // Since we write rollbacks not clean, there should be no clean action on the timeline + assertTrue(timeline.getTimelineOfActions( + CollectionUtils.createSet(HoodieTimeline.CLEAN_ACTION)).countInstants() == 0); + assertTrue(timeline.getCommitsTimeline().filterCompletedInstants().countInstants() == 2); + } + private Pair> testConsistencyCheck(HoodieTableMetaClient metaClient, String instantTime, boolean enableOptimisticConsistencyGuard) throws Exception { HoodieWriteConfig cfg = !enableOptimisticConsistencyGuard ? (getConfigBuilder().withAutoCommit(false) @@ -1746,6 +1929,8 @@ private HoodieWriteConfig getSmallInsertWriteConfig(int insertSplitSize, String .withCompactionConfig( HoodieCompactionConfig.newBuilder() .compactionSmallFileSize(smallFileSize) + // Set rollback to LAZY so no inflights are deleted + .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY) .insertSplitSize(insertSplitSize).build()) .withStorageConfig( HoodieStorageConfig.newBuilder() @@ -1767,4 +1952,15 @@ protected HoodieInstant createRequestedReplaceInstant(HoodieTableMetaClient meta return clusteringInstant; } + private HoodieWriteConfig getParallelWritingWriteConfig(HoodieFailedWritesCleaningPolicy cleaningPolicy) { + return getConfigBuilder() + .withEmbeddedTimelineServerEnabled(false) + .withCompactionConfig(HoodieCompactionConfig.newBuilder() + .withFailedWritesCleaningPolicy(cleaningPolicy) + .withAutoClean(false).build()) + .withTimelineLayoutVersion(1) + .withHeartbeatIntervalInMs(3 * 1000) + .withAutoCommit(false).build(); + } + } diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestTableSchemaEvolution.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestTableSchemaEvolution.java index ecf38d8a9017c..9bcacc9823b70 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestTableSchemaEvolution.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestTableSchemaEvolution.java @@ -155,7 +155,7 @@ public void testMORTable() throws Exception { metaClient.getArchivePath(), metaClient.getTableConfig().getPayloadClass(), VERSION_1); HoodieWriteConfig hoodieWriteConfig = getWriteConfig(TRIP_EXAMPLE_SCHEMA); - SparkRDDWriteClient client = getHoodieWriteClient(hoodieWriteConfig, false); + SparkRDDWriteClient client = getHoodieWriteClient(hoodieWriteConfig); // Initial inserts with TRIP_EXAMPLE_SCHEMA int numRecords = 10; @@ -184,13 +184,13 @@ public void testMORTable() throws Exception { // Insert with evolved schema is not allowed HoodieWriteConfig hoodieDevolvedWriteConfig = getWriteConfig(TRIP_EXAMPLE_SCHEMA_DEVOLVED); - client = getHoodieWriteClient(hoodieDevolvedWriteConfig, false); + client = getHoodieWriteClient(hoodieDevolvedWriteConfig); final List failedRecords = generateInsertsWithSchema("004", numRecords, TRIP_EXAMPLE_SCHEMA_DEVOLVED); try { // We cannot use insertBatch directly here because we want to insert records // with a devolved schema and insertBatch inserts records using the TRIP_EXMPLE_SCHEMA. writeBatch(client, "005", "004", Option.empty(), "003", numRecords, - (String s, Integer a) -> failedRecords, SparkRDDWriteClient::insert, false, 0, 0, 0); + (String s, Integer a) -> failedRecords, SparkRDDWriteClient::insert, false, 0, 0, 0, false); fail("Insert with devolved scheme should fail"); } catch (HoodieInsertException ex) { // no new commit @@ -213,13 +213,13 @@ public void testMORTable() throws Exception { // Insert with an evolved scheme is allowed HoodieWriteConfig hoodieEvolvedWriteConfig = getWriteConfig(TRIP_EXAMPLE_SCHEMA_EVOLVED); - client = getHoodieWriteClient(hoodieEvolvedWriteConfig, false); + client = getHoodieWriteClient(hoodieEvolvedWriteConfig); // We cannot use insertBatch directly here because we want to insert records // with a evolved schemaand insertBatch inserts records using the TRIP_EXMPLE_SCHEMA. final List evolvedRecords = generateInsertsWithSchema("005", numRecords, TRIP_EXAMPLE_SCHEMA_EVOLVED); writeBatch(client, "005", "004", Option.empty(), initCommitTime, numRecords, - (String s, Integer a) -> evolvedRecords, SparkRDDWriteClient::insert, false, 0, 0, 0); + (String s, Integer a) -> evolvedRecords, SparkRDDWriteClient::insert, false, 0, 0, 0, false); // new commit checkLatestDeltaCommit("005"); @@ -228,14 +228,14 @@ public void testMORTable() throws Exception { // Updates with evolved schema is allowed final List updateRecords = generateUpdatesWithSchema("006", numUpdateRecords, TRIP_EXAMPLE_SCHEMA_EVOLVED); writeBatch(client, "006", "005", Option.empty(), initCommitTime, - numUpdateRecords, (String s, Integer a) -> updateRecords, SparkRDDWriteClient::upsert, false, 0, 0, 0); + numUpdateRecords, (String s, Integer a) -> updateRecords, SparkRDDWriteClient::upsert, false, 0, 0, 0, false); // new commit checkLatestDeltaCommit("006"); checkReadRecords("000", 2 * numRecords); // Now even the original schema cannot be used for updates as it is devolved in relation to the // current schema of the dataset. - client = getHoodieWriteClient(hoodieWriteConfig, false); + client = getHoodieWriteClient(hoodieWriteConfig); try { updateBatch(hoodieWriteConfig, client, "007", "006", Option.empty(), initCommitTime, numUpdateRecords, SparkRDDWriteClient::upsert, false, false, 0, 0, 0); @@ -256,7 +256,7 @@ public void testMORTable() throws Exception { failedRecords.clear(); failedRecords.addAll(dataGen.generateInserts("007", numRecords)); writeBatch(client, "007", "006", Option.empty(), initCommitTime, numRecords, - (String s, Integer a) -> failedRecords, SparkRDDWriteClient::insert, true, numRecords, numRecords, 1); + (String s, Integer a) -> failedRecords, SparkRDDWriteClient::insert, true, numRecords, numRecords, 1, false); fail("Insert with original scheme should fail"); } catch (HoodieInsertException ex) { // no new commit @@ -278,7 +278,7 @@ public void testMORTable() throws Exception { checkLatestDeltaCommit("004"); // Updates with original schema are now allowed - client = getHoodieWriteClient(hoodieWriteConfig, false); + client = getHoodieWriteClient(hoodieWriteConfig); updateBatch(hoodieWriteConfig, client, "008", "004", Option.empty(), initCommitTime, numUpdateRecords, SparkRDDWriteClient::upsert, false, false, 0, 0, 0); // new commit @@ -300,7 +300,7 @@ public void testCopyOnWriteTable() throws Exception { metaClient.getArchivePath(), metaClient.getTableConfig().getPayloadClass(), VERSION_1); HoodieWriteConfig hoodieWriteConfig = getWriteConfig(TRIP_EXAMPLE_SCHEMA); - SparkRDDWriteClient client = getHoodieWriteClient(hoodieWriteConfig, false); + SparkRDDWriteClient client = getHoodieWriteClient(hoodieWriteConfig); // Initial inserts with TRIP_EXAMPLE_SCHEMA int numRecords = 10; @@ -324,13 +324,13 @@ public void testCopyOnWriteTable() throws Exception { // Insert with devolved schema is not allowed HoodieWriteConfig hoodieDevolvedWriteConfig = getWriteConfig(TRIP_EXAMPLE_SCHEMA_DEVOLVED); - client = getHoodieWriteClient(hoodieDevolvedWriteConfig, false); + client = getHoodieWriteClient(hoodieDevolvedWriteConfig); final List failedRecords = generateInsertsWithSchema("004", numRecords, TRIP_EXAMPLE_SCHEMA_DEVOLVED); try { // We cannot use insertBatch directly here because we want to insert records // with a devolved schema. writeBatch(client, "004", "003", Option.empty(), "003", numRecords, - (String s, Integer a) -> failedRecords, SparkRDDWriteClient::insert, true, numRecords, numRecords, 1); + (String s, Integer a) -> failedRecords, SparkRDDWriteClient::insert, true, numRecords, numRecords, 1, false); fail("Insert with devolved scheme should fail"); } catch (HoodieInsertException ex) { // no new commit @@ -354,12 +354,12 @@ public void testCopyOnWriteTable() throws Exception { // Insert with evolved scheme is allowed HoodieWriteConfig hoodieEvolvedWriteConfig = getWriteConfig(TRIP_EXAMPLE_SCHEMA_EVOLVED); - client = getHoodieWriteClient(hoodieEvolvedWriteConfig, false); + client = getHoodieWriteClient(hoodieEvolvedWriteConfig); final List evolvedRecords = generateInsertsWithSchema("004", numRecords, TRIP_EXAMPLE_SCHEMA_EVOLVED); // We cannot use insertBatch directly here because we want to insert records // with a evolved schema. writeBatch(client, "004", "003", Option.empty(), initCommitTime, numRecords, - (String s, Integer a) -> evolvedRecords, SparkRDDWriteClient::insert, true, numRecords, 2 * numRecords, 4); + (String s, Integer a) -> evolvedRecords, SparkRDDWriteClient::insert, true, numRecords, 2 * numRecords, 4, false); // new commit HoodieTimeline curTimeline = metaClient.reloadActiveTimeline().getCommitTimeline().filterCompletedInstants(); assertTrue(curTimeline.lastInstant().get().getTimestamp().equals("004")); @@ -368,12 +368,12 @@ public void testCopyOnWriteTable() throws Exception { // Updates with evolved schema is allowed final List updateRecords = generateUpdatesWithSchema("005", numUpdateRecords, TRIP_EXAMPLE_SCHEMA_EVOLVED); writeBatch(client, "005", "004", Option.empty(), initCommitTime, - numUpdateRecords, (String s, Integer a) -> updateRecords, SparkRDDWriteClient::upsert, true, numUpdateRecords, 2 * numRecords, 5); + numUpdateRecords, (String s, Integer a) -> updateRecords, SparkRDDWriteClient::upsert, true, numUpdateRecords, 2 * numRecords, 5, false); checkReadRecords("000", 2 * numRecords); // Now even the original schema cannot be used for updates as it is devolved // in relation to the current schema of the dataset. - client = getHoodieWriteClient(hoodieWriteConfig, false); + client = getHoodieWriteClient(hoodieWriteConfig); try { updateBatch(hoodieWriteConfig, client, "006", "005", Option.empty(), initCommitTime, numUpdateRecords, SparkRDDWriteClient::upsert, false, true, @@ -395,7 +395,7 @@ public void testCopyOnWriteTable() throws Exception { failedRecords.clear(); failedRecords.addAll(dataGen.generateInserts("006", numRecords)); writeBatch(client, "006", "005", Option.empty(), initCommitTime, numRecords, - (String s, Integer a) -> failedRecords, SparkRDDWriteClient::insert, true, numRecords, numRecords, 1); + (String s, Integer a) -> failedRecords, SparkRDDWriteClient::insert, true, numRecords, numRecords, 1, false); fail("Insert with original scheme should fail"); } catch (HoodieInsertException ex) { // no new commit diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/hbase/TestHBaseIndex.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/hbase/TestHBaseIndex.java index 85b5743f4a4a6..c99b79c900a17 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/hbase/TestHBaseIndex.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/hbase/TestHBaseIndex.java @@ -103,6 +103,9 @@ public static void clean() throws Exception { utility.deleteTable(TABLE_NAME); utility.shutdownMiniCluster(); } + if (spark != null) { + spark.close(); + } } @BeforeAll @@ -538,9 +541,9 @@ public void testsWriteStatusPartitioner() { final Map fileIdPartitionMap = index.mapFileWithInsertsToUniquePartition(writeStatusRDD); int numWriteStatusWithInserts = (int) index.getHBasePutAccessParallelism(writeStatusRDD)._2; JavaRDD partitionedRDD = writeStatusRDD.mapToPair(w -> new Tuple2<>(w.getFileId(), w)) - .partitionBy(new SparkHoodieHBaseIndex - .WriteStatusPartitioner(fileIdPartitionMap, - numWriteStatusWithInserts)).map(w -> w._2()); + .partitionBy(new SparkHoodieHBaseIndex + .WriteStatusPartitioner(fileIdPartitionMap, + numWriteStatusWithInserts)).map(w -> w._2()); assertEquals(numWriteStatusWithInserts, partitionedRDD.getNumPartitions()); int[] partitionIndexesBeforeRepartition = writeStatusRDD.partitions().stream().mapToInt(p -> p.index()).toArray(); assertEquals(parallelism, partitionIndexesBeforeRepartition.length); @@ -576,9 +579,9 @@ public void testsWriteStatusPartitionerWithNoInserts() { final Map fileIdPartitionMap = index.mapFileWithInsertsToUniquePartition(writeStatusRDD); int numWriteStatusWithInserts = (int) index.getHBasePutAccessParallelism(writeStatusRDD)._2; JavaRDD partitionedRDD = writeStatusRDD.mapToPair(w -> new Tuple2<>(w.getFileId(), w)) - .partitionBy(new SparkHoodieHBaseIndex - .WriteStatusPartitioner(fileIdPartitionMap, - numWriteStatusWithInserts)).map(w -> w._2()); + .partitionBy(new SparkHoodieHBaseIndex + .WriteStatusPartitioner(fileIdPartitionMap, + numWriteStatusWithInserts)).map(w -> w._2()); assertEquals(numWriteStatusWithInserts, partitionedRDD.getNumPartitions()); int[] partitionIndexesBeforeRepartition = writeStatusRDD.partitions().stream().mapToInt(p -> p.index()).toArray(); assertEquals(parallelism, partitionIndexesBeforeRepartition.length); @@ -749,4 +752,4 @@ private HoodieWriteConfig.Builder getConfigBuilder(int hbaseIndexBatchSize, bool .hbaseIndexGetBatchSize(hbaseIndexBatchSize).build()) .build()); } -} +} \ No newline at end of file diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieTimelineArchiveLog.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieTimelineArchiveLog.java index f2427cd9a2348..60f605c69b6bf 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieTimelineArchiveLog.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieTimelineArchiveLog.java @@ -408,11 +408,11 @@ public void testArchiveCommitCompactionNoHole() throws IOException { HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient); HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, table); - HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsAndCompactionTimeline(); + HoodieTimeline timeline = metaClient.getActiveTimeline().getWriteTimeline(); assertEquals(8, timeline.countInstants(), "Loaded 6 commits and the count should match"); boolean result = archiveLog.archiveIfRequired(context); assertTrue(result); - timeline = metaClient.getActiveTimeline().reload().getCommitsAndCompactionTimeline(); + timeline = metaClient.getActiveTimeline().reload().getWriteTimeline(); assertFalse(timeline.containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "100")), "Instants before oldest pending compaction can be removed"); assertEquals(7, timeline.countInstants(), diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/metadata/TestHoodieBackedMetadata.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/metadata/TestHoodieBackedMetadata.java index f822e85045dfd..3697ec1ba0d9b 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/metadata/TestHoodieBackedMetadata.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/metadata/TestHoodieBackedMetadata.java @@ -18,6 +18,8 @@ package org.apache.hudi.metadata; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; import org.apache.hudi.client.HoodieWriteResult; import org.apache.hudi.client.SparkRDDWriteClient; import org.apache.hudi.client.WriteStatus; @@ -52,9 +54,6 @@ import org.apache.hudi.table.HoodieSparkTable; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.testutils.HoodieClientTestHarness; - -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.Path; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import org.apache.spark.api.java.JavaRDD; @@ -610,13 +609,7 @@ public void testErrorCases() throws Exception { } try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true), true)) { - // Start the next commit which will rollback the previous one and also should update the metadata table by - // updating it with HoodieRollbackMetadata. String newCommitTime = client.startCommit(); - - // Dangling commit but metadata should be valid at this time - validateMetadata(client); - // Next insert List records = dataGen.generateInserts(newCommitTime, 5); List writeStatuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect(); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestCleaner.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestCleaner.java index 4fff08abfbcef..1119f2665b3aa 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestCleaner.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestCleaner.java @@ -26,6 +26,7 @@ import org.apache.hudi.avro.model.HoodieCleanerPlan; import org.apache.hudi.avro.model.HoodieCompactionPlan; import org.apache.hudi.avro.model.HoodieFileStatus; +import org.apache.hudi.avro.model.HoodieRollbackMetadata; import org.apache.hudi.client.SparkRDDWriteClient; import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.HoodieCleanStat; @@ -38,6 +39,7 @@ import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieCleaningPolicy; import org.apache.hudi.common.model.HoodieCommitMetadata; +import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy; import org.apache.hudi.common.model.HoodieFileGroup; import org.apache.hudi.common.model.HoodieFileGroupId; import org.apache.hudi.common.model.HoodieRecord; @@ -96,6 +98,7 @@ import java.util.Set; import java.util.TreeSet; import java.util.UUID; +import java.util.concurrent.TimeUnit; import java.util.function.Predicate; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -104,6 +107,7 @@ import static org.apache.hudi.common.testutils.HoodieTestTable.makeNewCommitTime; import static org.apache.hudi.common.testutils.HoodieTestUtils.DEFAULT_PARTITION_PATHS; import static org.apache.hudi.testutils.Assertions.assertNoWriteErrors; +import static org.awaitility.Awaitility.await; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertNull; @@ -126,7 +130,7 @@ public class TestCleaner extends HoodieClientTestBase { * @param insertFn Insertion API for testing * @throws Exception in case of error */ - private void insertFirstBigBatchForClientCleanerTest(HoodieWriteConfig cfg, SparkRDDWriteClient client, + private Pair> insertFirstBigBatchForClientCleanerTest(HoodieWriteConfig cfg, SparkRDDWriteClient client, Function2, String, Integer> recordGenFunction, Function3, SparkRDDWriteClient, JavaRDD, String> insertFn, HoodieCleaningPolicy cleaningPolicy) throws Exception { @@ -140,10 +144,9 @@ private void insertFirstBigBatchForClientCleanerTest(HoodieWriteConfig cfg, Spar List records = recordGenFunction.apply(newCommitTime, BIG_BATCH_INSERT_SIZE); JavaRDD writeRecords = jsc.parallelize(records, 5); - List statuses = insertFn.apply(client, writeRecords, newCommitTime).collect(); + JavaRDD statuses = insertFn.apply(client, writeRecords, newCommitTime); // Verify there are no errors - assertNoWriteErrors(statuses); - + assertNoWriteErrors(statuses.collect()); // verify that there is a commit metaClient = HoodieTableMetaClient.reload(metaClient); HoodieTimeline timeline = new HoodieActiveTimeline(metaClient).getCommitTimeline(); @@ -151,13 +154,49 @@ private void insertFirstBigBatchForClientCleanerTest(HoodieWriteConfig cfg, Spar // Should have 100 records in table (check using Index), all in locations marked at commit HoodieTable table = HoodieSparkTable.create(client.getConfig(), context, metaClient); - assertFalse(table.getCompletedCommitsTimeline().empty()); + if (client.getConfig().shouldAutoCommit()) { + assertFalse(table.getCompletedCommitsTimeline().empty()); + } // We no longer write empty cleaner plans when there is nothing to be cleaned. assertTrue(table.getCompletedCleanTimeline().empty()); - HoodieIndex index = SparkHoodieIndex.createIndex(cfg); - List taggedRecords = ((JavaRDD) index.tagLocation(jsc.parallelize(records, 1), context, table)).collect(); - checkTaggedRecords(taggedRecords, newCommitTime); + if (client.getConfig().shouldAutoCommit()) { + HoodieIndex index = SparkHoodieIndex.createIndex(cfg); + List taggedRecords = ((JavaRDD) index.tagLocation(jsc.parallelize(records, 1), context, table)).collect(); + checkTaggedRecords(taggedRecords, newCommitTime); + } + return Pair.of(newCommitTime, statuses); + } + + /** + * Helper method to do first batch of insert for clean by versions/commits tests. + * + * @param cfg Hoodie Write Config + * @param client Hoodie Client + * @param recordGenFunction Function to generate records for insertion + * @param insertFn Insertion API for testing + * @throws Exception in case of error + */ + private Pair> insertFirstFailedBigBatchForClientCleanerTest(HoodieWriteConfig cfg, SparkRDDWriteClient client, + Function2, String, Integer> recordGenFunction, + Function3, SparkRDDWriteClient, JavaRDD, String> insertFn, + HoodieCleaningPolicy cleaningPolicy) throws Exception { + + /* + * do a big insert (this is basically same as insert part of upsert, just adding it here so we can catch breakages + * in insert(), if the implementation diverges.) + */ + String newCommitTime = client.startCommit(); + + List records = recordGenFunction.apply(newCommitTime, BIG_BATCH_INSERT_SIZE); + JavaRDD writeRecords = jsc.parallelize(records, 5); + + JavaRDD statuses = insertFn.apply(client, writeRecords, newCommitTime); + // Verify there are no errors + assertNoWriteErrors(statuses.collect()); + // Don't invoke commit to simulate failed write + client.getHeartbeatClient().stop(newCommitTime); + return Pair.of(newCommitTime, statuses); } /** @@ -168,6 +207,14 @@ public void testInsertAndCleanByVersions() throws Exception { testInsertAndCleanByVersions(SparkRDDWriteClient::insert, SparkRDDWriteClient::upsert, false); } + /** + * Test Clean-Failed-Writes when Cleaning policy is by VERSIONS using insert/upsert API. + */ + @Test + public void testInsertAndCleanFailedWritesByVersions() throws Exception { + testInsertAndCleanFailedWritesByVersions(SparkRDDWriteClient::insert, false); + } + /** * Test Clean-By-Versions using prepped versions of insert/upsert API. */ @@ -323,7 +370,7 @@ private void testInsertAndCleanByVersions( } /** - * Test Clean-By-Versions using insert/upsert API. + * Test Clean-By-Commits using insert/upsert API. */ @Test public void testInsertAndCleanByCommits() throws Exception { @@ -331,7 +378,15 @@ public void testInsertAndCleanByCommits() throws Exception { } /** - * Test Clean-By-Versions using prepped version of insert/upsert API. + * Test Clean-By-Commits using insert/upsert API. + */ + @Test + public void testFailedInsertAndCleanByCommits() throws Exception { + testFailedInsertAndCleanByCommits(SparkRDDWriteClient::insert, false); + } + + /** + * Test Clean-By-Commits using prepped version of insert/upsert API. */ @Test public void testInsertPreppedAndCleanByCommits() throws Exception { @@ -339,7 +394,7 @@ public void testInsertPreppedAndCleanByCommits() throws Exception { } /** - * Test Clean-By-Versions using prepped versions of bulk-insert/upsert API. + * Test Clean-By-Commits using prepped versions of bulk-insert/upsert API. */ @Test public void testBulkInsertPreppedAndCleanByCommits() throws Exception { @@ -349,7 +404,7 @@ public void testBulkInsertPreppedAndCleanByCommits() throws Exception { } /** - * Test Clean-By-Versions using bulk-insert/upsert API. + * Test Clean-By-Commits using bulk-insert/upsert API. */ @Test public void testBulkInsertAndCleanByCommits() throws Exception { @@ -431,6 +486,64 @@ private void testInsertAndCleanByCommits( }); } + /** + * Test Helper for Cleaning failed commits by commits logic from HoodieWriteClient API perspective. + * + * @param insertFn Insert API to be tested + * @param isPreppedAPI Flag to indicate if a prepped-version is used. If true, a wrapper function will be used during + * record generation to also tag the regards (de-dupe is implicit as we use uniq record-gen APIs) + * @throws Exception in case of errors + */ + private void testFailedInsertAndCleanByCommits( + Function3, SparkRDDWriteClient, JavaRDD, String> insertFn, boolean isPreppedAPI) + throws Exception { + int maxCommits = 3; // keep upto 3 commits from the past + HoodieWriteConfig cfg = getConfigBuilder() + .withAutoCommit(false) + .withHeartbeatIntervalInMs(3000) + .withCompactionConfig(HoodieCompactionConfig.newBuilder() + .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY) + .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(maxCommits).build()) + .withParallelism(1, 1).withBulkInsertParallelism(1).withFinalizeWriteParallelism(1).withDeleteParallelism(1) + .withConsistencyGuardConfig(ConsistencyGuardConfig.newBuilder().withConsistencyCheckEnabled(true).build()) + .build(); + SparkRDDWriteClient client = getHoodieWriteClient(cfg); + + final Function2, String, Integer> recordInsertGenWrappedFunction = + generateWrapRecordsFn(isPreppedAPI, cfg, dataGen::generateInserts); + + Pair> result = insertFirstBigBatchForClientCleanerTest(cfg, client, recordInsertGenWrappedFunction, insertFn, + HoodieCleaningPolicy.KEEP_LATEST_COMMITS); + client.commit(result.getLeft(), result.getRight()); + + HoodieTable table = HoodieSparkTable.create(client.getConfig(), context, metaClient); + assertTrue(table.getCompletedCleanTimeline().empty()); + + insertFirstFailedBigBatchForClientCleanerTest(cfg, client, recordInsertGenWrappedFunction, insertFn, + HoodieCleaningPolicy.KEEP_LATEST_COMMITS); + + insertFirstFailedBigBatchForClientCleanerTest(cfg, client, recordInsertGenWrappedFunction, insertFn, + HoodieCleaningPolicy.KEEP_LATEST_COMMITS); + + Pair> ret = + insertFirstFailedBigBatchForClientCleanerTest(cfg, client, recordInsertGenWrappedFunction, insertFn, + HoodieCleaningPolicy.KEEP_LATEST_COMMITS); + // Await till enough time passes such that the last failed commits heartbeats are expired + await().atMost(10, TimeUnit.SECONDS).until(() -> client.getHeartbeatClient() + .isHeartbeatExpired(ret.getLeft())); + List cleanStats = runCleaner(cfg); + assertEquals(0, cleanStats.size(), "Must not clean any files"); + HoodieActiveTimeline timeline = metaClient.reloadActiveTimeline(); + assertTrue(timeline.getTimelineOfActions( + CollectionUtils.createSet(HoodieTimeline.ROLLBACK_ACTION)).filterCompletedInstants().countInstants() == 3); + Option rolleBackInstantForFailedCommit = timeline.getTimelineOfActions( + CollectionUtils.createSet(HoodieTimeline.ROLLBACK_ACTION)).filterCompletedInstants().lastInstant(); + HoodieRollbackMetadata rollbackMetadata = TimelineMetadataUtils.deserializeAvroMetadata( + timeline.getInstantDetails(rolleBackInstantForFailedCommit.get()).get(), HoodieRollbackMetadata.class); + // Rollback of one of the failed writes should have deleted 3 files + assertEquals(3, rollbackMetadata.getTotalFilesDeleted()); + } + /** * Helper to run cleaner and collect Clean Stats. * @@ -980,6 +1093,7 @@ public void testKeepLatestCommits(boolean simulateFailureRetry, boolean enableIn .withMetadataConfig(HoodieMetadataConfig.newBuilder().withAssumeDatePartitioning(true).build()) .withCompactionConfig(HoodieCompactionConfig.newBuilder() .withIncrementalCleaningMode(enableIncrementalClean) + .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.EAGER) .withCleanBootstrapBaseFileEnabled(enableBootstrapSourceClean) .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(2).build()) .build(); @@ -1246,6 +1360,69 @@ public void testCleanPreviousCorruptedCleanFiles() throws IOException { assertEquals(0, cleanStats.size(), "Must not clean any files"); } + /** + * Test Helper for cleaning failed writes by versions logic from HoodieWriteClient API perspective. + * + * @param insertFn Insert API to be tested + * @param isPreppedAPI Flag to indicate if a prepped-version is used. If true, a wrapper function will be used during + * record generation to also tag the regards (de-dupe is implicit as we use uniq record-gen APIs) + * @throws Exception in case of errors + */ + private void testInsertAndCleanFailedWritesByVersions( + Function3, SparkRDDWriteClient, JavaRDD, String> insertFn, boolean isPreppedAPI) + throws Exception { + int maxVersions = 3; // keep upto 3 versions for each file + HoodieWriteConfig cfg = getConfigBuilder() + .withAutoCommit(false) + .withHeartbeatIntervalInMs(3000) + .withCompactionConfig(HoodieCompactionConfig.newBuilder() + .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY) + .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS).retainFileVersions(maxVersions).build()) + .withParallelism(1, 1).withBulkInsertParallelism(1).withFinalizeWriteParallelism(1).withDeleteParallelism(1) + .withConsistencyGuardConfig(ConsistencyGuardConfig.newBuilder().withConsistencyCheckEnabled(true).build()) + .build(); + try (SparkRDDWriteClient client = getHoodieWriteClient(cfg);) { + + final Function2, String, Integer> recordInsertGenWrappedFunction = + generateWrapRecordsFn(isPreppedAPI, cfg, dataGen::generateInserts); + + Pair> result = insertFirstBigBatchForClientCleanerTest(cfg, client, recordInsertGenWrappedFunction, insertFn, + HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS); + + client.commit(result.getLeft(), result.getRight()); + + HoodieTable table = HoodieSparkTable.create(client.getConfig(), context, metaClient); + + assertTrue(table.getCompletedCleanTimeline().empty()); + + insertFirstFailedBigBatchForClientCleanerTest(cfg, client, recordInsertGenWrappedFunction, insertFn, + HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS); + + insertFirstFailedBigBatchForClientCleanerTest(cfg, client, recordInsertGenWrappedFunction, insertFn, + HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS); + + Pair> ret = + insertFirstFailedBigBatchForClientCleanerTest(cfg, client, recordInsertGenWrappedFunction, insertFn, + HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS); + + // Await till enough time passes such that the last failed commits heartbeats are expired + await().atMost(10, TimeUnit.SECONDS).until(() -> client.getHeartbeatClient() + .isHeartbeatExpired(ret.getLeft())); + + List cleanStats = runCleaner(cfg); + assertEquals(0, cleanStats.size(), "Must not clean any files"); + HoodieActiveTimeline timeline = metaClient.reloadActiveTimeline(); + assertTrue(timeline.getTimelineOfActions( + CollectionUtils.createSet(HoodieTimeline.ROLLBACK_ACTION)).filterCompletedInstants().countInstants() == 3); + Option rolleBackInstantForFailedCommit = timeline.getTimelineOfActions( + CollectionUtils.createSet(HoodieTimeline.ROLLBACK_ACTION)).filterCompletedInstants().lastInstant(); + HoodieRollbackMetadata rollbackMetadata = TimelineMetadataUtils.deserializeAvroMetadata( + timeline.getInstantDetails(rolleBackInstantForFailedCommit.get()).get(), HoodieRollbackMetadata.class); + // Rollback of one of the failed writes should have deleted 3 files + assertEquals(3, rollbackMetadata.getTotalFilesDeleted()); + } + } + /** * Common test method for validating pending compactions. * diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/CompactionTestBase.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/CompactionTestBase.java index b8bccbc36f82e..6c8a54dab9070 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/CompactionTestBase.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/CompactionTestBase.java @@ -262,7 +262,7 @@ protected List getCurrentLatestBaseFiles(HoodieTable table) thro protected List getCurrentLatestFileSlices(HoodieTable table) { HoodieTableFileSystemView view = new HoodieTableFileSystemView(table.getMetaClient(), - table.getMetaClient().getActiveTimeline().reload().getCommitsAndCompactionTimeline()); + table.getMetaClient().getActiveTimeline().reload().getWriteTimeline()); return Arrays.stream(HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS) .flatMap(view::getLatestFileSlices).collect(Collectors.toList()); } diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/TestAsyncCompaction.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/TestAsyncCompaction.java index 6e8326c6ad0a3..79c415a4bc268 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/TestAsyncCompaction.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/TestAsyncCompaction.java @@ -60,7 +60,7 @@ private HoodieWriteConfig getConfig(Boolean autoCommit) { public void testRollbackForInflightCompaction() throws Exception { // Rollback inflight compaction HoodieWriteConfig cfg = getConfig(false); - try (SparkRDDWriteClient client = getHoodieWriteClient(cfg, true);) { + try (SparkRDDWriteClient client = getHoodieWriteClient(cfg);) { HoodieReadClient readClient = getHoodieReadClient(cfg.getBasePath()); String firstInstantTime = "001"; String secondInstantTime = "004"; @@ -120,7 +120,7 @@ public void testRollbackInflightIngestionWithPendingCompaction() throws Exceptio int numRecs = 2000; - try (SparkRDDWriteClient client = getHoodieWriteClient(cfg, true);) { + try (SparkRDDWriteClient client = getHoodieWriteClient(cfg);) { HoodieReadClient readClient = getHoodieReadClient(cfg.getBasePath()); List records = dataGen.generateInserts(firstInstantTime, numRecs); records = runNextDeltaCommits(client, readClient, Arrays.asList(firstInstantTime, secondInstantTime), records, cfg, true, @@ -162,7 +162,7 @@ public void testRollbackInflightIngestionWithPendingCompaction() throws Exceptio public void testInflightCompaction() throws Exception { // There is inflight compaction. Subsequent compaction run must work correctly HoodieWriteConfig cfg = getConfig(true); - try (SparkRDDWriteClient client = getHoodieWriteClient(cfg, true);) { + try (SparkRDDWriteClient client = getHoodieWriteClient(cfg);) { HoodieReadClient readClient = getHoodieReadClient(cfg.getBasePath()); String firstInstantTime = "001"; String secondInstantTime = "004"; @@ -195,7 +195,7 @@ public void testInflightCompaction() throws Exception { public void testScheduleIngestionBeforePendingCompaction() throws Exception { // Case: Failure case. Latest pending compaction instant time must be earlier than this instant time HoodieWriteConfig cfg = getConfig(false); - SparkRDDWriteClient client = getHoodieWriteClient(cfg, true); + SparkRDDWriteClient client = getHoodieWriteClient(cfg); HoodieReadClient readClient = getHoodieReadClient(cfg.getBasePath()); String firstInstantTime = "001"; @@ -226,7 +226,7 @@ public void testScheduleCompactionAfterPendingIngestion() throws Exception { // Case: Failure case. Earliest ingestion inflight instant time must be later than compaction time HoodieWriteConfig cfg = getConfig(false); - SparkRDDWriteClient client = getHoodieWriteClient(cfg, true); + SparkRDDWriteClient client = getHoodieWriteClient(cfg); HoodieReadClient readClient = getHoodieReadClient(cfg.getBasePath()); String firstInstantTime = "001"; @@ -258,7 +258,7 @@ public void testScheduleCompactionWithOlderOrSameTimestamp() throws Exception { // Case: Failure case. Earliest ingestion inflight instant time must be later than compaction time HoodieWriteConfig cfg = getConfig(false); - SparkRDDWriteClient client = getHoodieWriteClient(cfg, true); + SparkRDDWriteClient client = getHoodieWriteClient(cfg); HoodieReadClient readClient = getHoodieReadClient(cfg.getBasePath()); final String firstInstantTime = "001"; @@ -293,7 +293,7 @@ public void testScheduleCompactionWithOlderOrSameTimestamp() throws Exception { public void testCompactionAfterTwoDeltaCommits() throws Exception { // No Delta Commits after compaction request HoodieWriteConfig cfg = getConfig(true); - try (SparkRDDWriteClient client = getHoodieWriteClient(cfg, true);) { + try (SparkRDDWriteClient client = getHoodieWriteClient(cfg);) { HoodieReadClient readClient = getHoodieReadClient(cfg.getBasePath()); String firstInstantTime = "001"; String secondInstantTime = "004"; @@ -314,7 +314,7 @@ public void testCompactionAfterTwoDeltaCommits() throws Exception { public void testInterleavedCompaction() throws Exception { // Case: Two delta commits before and after compaction schedule HoodieWriteConfig cfg = getConfig(true); - try (SparkRDDWriteClient client = getHoodieWriteClient(cfg, true);) { + try (SparkRDDWriteClient client = getHoodieWriteClient(cfg);) { HoodieReadClient readClient = getHoodieReadClient(cfg.getBasePath()); String firstInstantTime = "001"; String secondInstantTime = "004"; @@ -342,7 +342,7 @@ public void testInterleavedCompaction() throws Exception { public void testCompactionOnReplacedFiles() throws Exception { // Schedule a compaction. Replace those file groups and ensure compaction completes successfully. HoodieWriteConfig cfg = getConfig(true); - try (SparkRDDWriteClient client = getHoodieWriteClient(cfg, true);) { + try (SparkRDDWriteClient client = getHoodieWriteClient(cfg);) { HoodieReadClient readClient = getHoodieReadClient(cfg.getBasePath()); String firstInstantTime = "001"; String secondInstantTime = "004"; diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/TestInlineCompaction.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/TestInlineCompaction.java index 97d287592b4b2..823d651aa1589 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/TestInlineCompaction.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/TestInlineCompaction.java @@ -60,7 +60,7 @@ public void testCompactionIsNotScheduledEarly() throws Exception { HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setConf(hadoopConf).setBasePath(cfg.getBasePath()).build(); // Then: ensure no compaction is executedm since there are only 2 delta commits - assertEquals(2, metaClient.getActiveTimeline().getCommitsAndCompactionTimeline().countInstants()); + assertEquals(2, metaClient.getActiveTimeline().getWriteTimeline().countInstants()); } } @@ -82,7 +82,7 @@ public void testSuccessfulCompactionBasedOnNumCommits() throws Exception { // Then: ensure the file slices are compacted as per policy metaClient = HoodieTableMetaClient.builder().setConf(hadoopConf).setBasePath(cfg.getBasePath()).build(); - assertEquals(4, metaClient.getActiveTimeline().getCommitsAndCompactionTimeline().countInstants()); + assertEquals(4, metaClient.getActiveTimeline().getWriteTimeline().countInstants()); assertEquals(HoodieTimeline.COMMIT_ACTION, metaClient.getActiveTimeline().lastInstant().get().getAction()); } } @@ -105,7 +105,7 @@ public void testSuccessfulCompactionBasedOnTime() throws Exception { // Then: ensure the file slices are compacted as per policy metaClient = HoodieTableMetaClient.builder().setConf(hadoopConf).setBasePath(cfg.getBasePath()).build(); - assertEquals(3, metaClient.getActiveTimeline().getCommitsAndCompactionTimeline().countInstants()); + assertEquals(3, metaClient.getActiveTimeline().getWriteTimeline().countInstants()); assertEquals(HoodieTimeline.COMMIT_ACTION, metaClient.getActiveTimeline().lastInstant().get().getAction()); } } @@ -125,14 +125,14 @@ public void testSuccessfulCompactionBasedOnNumOrTime() throws Exception { createNextDeltaCommit(finalInstant, dataGen.generateUpdates(finalInstant, 10), writeClient, metaClient, cfg, false); metaClient = HoodieTableMetaClient.builder().setConf(hadoopConf).setBasePath(cfg.getBasePath()).build(); - assertEquals(4, metaClient.getActiveTimeline().getCommitsAndCompactionTimeline().countInstants()); + assertEquals(4, metaClient.getActiveTimeline().getWriteTimeline().countInstants()); // 4th commit, that will trigger compaction because reach the time elapsed metaClient = HoodieTableMetaClient.builder().setConf(hadoopConf).setBasePath(cfg.getBasePath()).build(); finalInstant = HoodieActiveTimeline.createNewInstantTime(20000); createNextDeltaCommit(finalInstant, dataGen.generateUpdates(finalInstant, 10), writeClient, metaClient, cfg, false); metaClient = HoodieTableMetaClient.builder().setConf(hadoopConf).setBasePath(cfg.getBasePath()).build(); - assertEquals(6, metaClient.getActiveTimeline().getCommitsAndCompactionTimeline().countInstants()); + assertEquals(6, metaClient.getActiveTimeline().getWriteTimeline().countInstants()); } } @@ -148,14 +148,14 @@ public void testSuccessfulCompactionBasedOnNumAndTime() throws Exception { HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setConf(hadoopConf).setBasePath(cfg.getBasePath()).build(); // Then: ensure no compaction is executedm since there are only 3 delta commits - assertEquals(3, metaClient.getActiveTimeline().getCommitsAndCompactionTimeline().countInstants()); + assertEquals(3, metaClient.getActiveTimeline().getWriteTimeline().countInstants()); // 4th commit, that will trigger compaction metaClient = HoodieTableMetaClient.builder().setConf(hadoopConf).setBasePath(cfg.getBasePath()).build(); String finalInstant = HoodieActiveTimeline.createNewInstantTime(20000); createNextDeltaCommit(finalInstant, dataGen.generateUpdates(finalInstant, 10), writeClient, metaClient, cfg, false); metaClient = HoodieTableMetaClient.builder().setConf(hadoopConf).setBasePath(cfg.getBasePath()).build(); - assertEquals(5, metaClient.getActiveTimeline().getCommitsAndCompactionTimeline().countInstants()); + assertEquals(5, metaClient.getActiveTimeline().getWriteTimeline().countInstants()); } } @@ -189,7 +189,7 @@ public void testCompactionRetryOnFailureBasedOnNumCommits() throws Exception { // Then: 1 delta commit is done, the failed compaction is retried metaClient = HoodieTableMetaClient.builder().setConf(hadoopConf).setBasePath(cfg.getBasePath()).build(); - assertEquals(4, metaClient.getActiveTimeline().getCommitsAndCompactionTimeline().countInstants()); + assertEquals(4, metaClient.getActiveTimeline().getWriteTimeline().countInstants()); assertEquals(instantTime2, metaClient.getActiveTimeline().getCommitTimeline().filterCompletedInstants().firstInstant().get().getTimestamp()); } @@ -225,7 +225,7 @@ public void testCompactionRetryOnFailureBasedOnTime() throws Exception { // Then: 1 delta commit is done, the failed compaction is retried metaClient = HoodieTableMetaClient.builder().setConf(hadoopConf).setBasePath(cfg.getBasePath()).build(); - assertEquals(4, metaClient.getActiveTimeline().getCommitsAndCompactionTimeline().countInstants()); + assertEquals(4, metaClient.getActiveTimeline().getWriteTimeline().countInstants()); assertEquals(instantTime, metaClient.getActiveTimeline().getCommitTimeline().filterCompletedInstants().firstInstant().get().getTimestamp()); } @@ -262,7 +262,7 @@ public void testCompactionRetryOnFailureBasedOnNumAndTime() throws Exception { // Then: 1 delta commit is done, the failed compaction is retried metaClient = HoodieTableMetaClient.builder().setConf(hadoopConf).setBasePath(cfg.getBasePath()).build(); - assertEquals(4, metaClient.getActiveTimeline().getCommitsAndCompactionTimeline().countInstants()); + assertEquals(4, metaClient.getActiveTimeline().getWriteTimeline().countInstants()); assertEquals(instantTime, metaClient.getActiveTimeline().getCommitTimeline().filterCompletedInstants().firstInstant().get().getTimestamp()); } } diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/upgrade/TestUpgradeDowngrade.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/upgrade/TestUpgradeDowngrade.java index 6a292f5f1f53f..e2416bb52def5 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/upgrade/TestUpgradeDowngrade.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/upgrade/TestUpgradeDowngrade.java @@ -18,6 +18,10 @@ package org.apache.hudi.table.upgrade; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileUtil; +import org.apache.hadoop.fs.Path; import org.apache.hudi.client.SparkRDDWriteClient; import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.model.FileSlice; @@ -38,11 +42,6 @@ import org.apache.hudi.testutils.Assertions; import org.apache.hudi.testutils.HoodieClientTestBase; import org.apache.hudi.testutils.HoodieClientTestUtils; - -import org.apache.hadoop.fs.FSDataInputStream; -import org.apache.hadoop.fs.FSDataOutputStream; -import org.apache.hadoop.fs.FileUtil; -import org.apache.hadoop.fs.Path; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; @@ -291,7 +290,7 @@ private List triggerCommit(String newCommitTime, HoodieTableType t params.put(HOODIE_TABLE_TYPE_PROP_NAME, HoodieTableType.MERGE_ON_READ.name()); } HoodieWriteConfig cfg = getConfigBuilder().withAutoCommit(false).withRollbackUsingMarkers(enableMarkedBasedRollback).withProps(params).build(); - SparkRDDWriteClient client = getHoodieWriteClient(cfg, true); + SparkRDDWriteClient client = getHoodieWriteClient(cfg); client.startCommitWithTime(newCommitTime); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/FunctionalTestHarness.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/FunctionalTestHarness.java index 57767d519c398..1db1b7f468128 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/FunctionalTestHarness.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/FunctionalTestHarness.java @@ -56,7 +56,7 @@ public class FunctionalTestHarness implements SparkProvider, DFSProvider, HoodieMetaClientProvider, HoodieWriteClientProvider { - private static transient SparkSession spark; + protected static transient SparkSession spark; private static transient SQLContext sqlContext; private static transient JavaSparkContext jsc; protected static transient HoodieSparkEngineContext context; @@ -126,7 +126,7 @@ public HoodieTableMetaClient getHoodieMetaClient(Configuration hadoopConf, Strin @Override public SparkRDDWriteClient getHoodieWriteClient(HoodieWriteConfig cfg) throws IOException { - return new SparkRDDWriteClient(context(), cfg, false); + return new SparkRDDWriteClient(context(), cfg); } @BeforeEach diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestBase.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestBase.java index 1104631a3c30a..1386edcfee1aa 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestBase.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestBase.java @@ -23,6 +23,7 @@ import org.apache.hudi.common.HoodieCleanStat; import org.apache.hudi.common.fs.ConsistencyGuardConfig; import org.apache.hudi.common.model.EmptyHoodieRecordPayload; +import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodiePartitionMetadata; import org.apache.hudi.common.model.HoodieRecord; @@ -105,17 +106,30 @@ public HoodieWriteConfig.Builder getConfigBuilder() { return getConfigBuilder(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA); } + /** + * Get Config builder with default configs set. + * + * @return Config Builder + */ + public HoodieWriteConfig.Builder getConfigBuilder(HoodieFailedWritesCleaningPolicy cleaningPolicy) { + return getConfigBuilder(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA, IndexType.BLOOM, cleaningPolicy); + } + /** * Get Config builder with default configs set. * * @return Config Builder */ public HoodieWriteConfig.Builder getConfigBuilder(IndexType indexType) { - return getConfigBuilder(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA, indexType); + return getConfigBuilder(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA, indexType, HoodieFailedWritesCleaningPolicy.EAGER); } public HoodieWriteConfig.Builder getConfigBuilder(String schemaStr) { - return getConfigBuilder(schemaStr, IndexType.BLOOM); + return getConfigBuilder(schemaStr, IndexType.BLOOM, HoodieFailedWritesCleaningPolicy.EAGER); + } + + public HoodieWriteConfig.Builder getConfigBuilder(String schemaStr, IndexType indexType) { + return getConfigBuilder(schemaStr, indexType, HoodieFailedWritesCleaningPolicy.EAGER); } /** @@ -123,13 +137,15 @@ public HoodieWriteConfig.Builder getConfigBuilder(String schemaStr) { * * @return Config Builder */ - public HoodieWriteConfig.Builder getConfigBuilder(String schemaStr, IndexType indexType) { + public HoodieWriteConfig.Builder getConfigBuilder(String schemaStr, IndexType indexType, + HoodieFailedWritesCleaningPolicy cleaningPolicy) { return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(schemaStr) .withParallelism(2, 2).withBulkInsertParallelism(2).withFinalizeWriteParallelism(2).withDeleteParallelism(2) .withTimelineLayoutVersion(TimelineLayoutVersion.CURR_VERSION) .withWriteStatusClass(MetadataMergeWriteStatus.class) .withConsistencyGuardConfig(ConsistencyGuardConfig.newBuilder().withConsistencyCheckEnabled(true).build()) - .withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024).build()) + .withCompactionConfig(HoodieCompactionConfig.newBuilder().withFailedWritesCleaningPolicy(cleaningPolicy) + .compactionSmallFileSize(1024 * 1024).build()) .withStorageConfig(HoodieStorageConfig.newBuilder().hfileMaxFileSize(1024 * 1024).parquetMaxFileSize(1024 * 1024).build()) .forTable("test-trip-table") .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(indexType).build()) @@ -308,7 +324,7 @@ public JavaRDD insertFirstBatch(HoodieWriteConfig writeConfig, Spar generateWrapRecordsFn(isPreppedAPI, writeConfig, dataGen::generateInserts); return writeBatch(client, newCommitTime, initCommitTime, Option.empty(), initCommitTime, numRecordsInThisCommit, - recordGenFunction, writeFn, assertForCommit, expRecordsInThisCommit, expRecordsInThisCommit, 1); + recordGenFunction, writeFn, assertForCommit, expRecordsInThisCommit, expRecordsInThisCommit, 1, false); } /** @@ -336,7 +352,7 @@ public JavaRDD insertBatch(HoodieWriteConfig writeConfig, SparkRDDW generateWrapRecordsFn(isPreppedAPI, writeConfig, dataGen::generateInserts); return writeBatch(client, newCommitTime, initCommitTime, Option.empty(), initCommitTime, numRecordsInThisCommit, - recordGenFunction, writeFn, assertForCommit, expRecordsInThisCommit, expTotalRecords, expTotalCommits); + recordGenFunction, writeFn, assertForCommit, expRecordsInThisCommit, expTotalRecords, expTotalCommits, false); } /** @@ -368,7 +384,7 @@ public JavaRDD updateBatch(HoodieWriteConfig writeConfig, SparkRDDW return writeBatch(client, newCommitTime, prevCommitTime, commitTimesBetweenPrevAndNew, initCommitTime, numRecordsInThisCommit, recordGenFunction, writeFn, assertForCommit, expRecordsInThisCommit, expTotalRecords, - expTotalCommits); + expTotalCommits, false); } /** @@ -416,13 +432,14 @@ public JavaRDD deleteBatch(HoodieWriteConfig writeConfig, SparkRDDW * @param expRecordsInThisCommit Expected number of records in this commit * @param expTotalRecords Expected number of records when scanned * @param expTotalCommits Expected number of commits (including this commit) + * @param doCommit * @throws Exception in case of error */ public JavaRDD writeBatch(SparkRDDWriteClient client, String newCommitTime, String prevCommitTime, - Option> commitTimesBetweenPrevAndNew, String initCommitTime, int numRecordsInThisCommit, - Function2, String, Integer> recordGenFunction, - Function3, SparkRDDWriteClient, JavaRDD, String> writeFn, - boolean assertForCommit, int expRecordsInThisCommit, int expTotalRecords, int expTotalCommits) throws Exception { + Option> commitTimesBetweenPrevAndNew, String initCommitTime, int numRecordsInThisCommit, + Function2, String, Integer> recordGenFunction, + Function3, SparkRDDWriteClient, JavaRDD, String> writeFn, + boolean assertForCommit, int expRecordsInThisCommit, int expTotalRecords, int expTotalCommits, boolean doCommit) throws Exception { // Write 1 (only inserts) client.startCommitWithTime(newCommitTime); @@ -434,6 +451,9 @@ public JavaRDD writeBatch(SparkRDDWriteClient client, String newCom List statuses = result.collect(); assertNoWriteErrors(statuses); + if (doCommit) { + client.commit(newCommitTime, result); + } // check the partition metadata is written out assertPartitionMetadataForRecords(records, fs); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestHarness.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestHarness.java index f3febaba03580..75c139e43dfff 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestHarness.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestHarness.java @@ -327,21 +327,17 @@ private void initFileSystemWithConfiguration(Configuration configuration) { } } - public SparkRDDWriteClient getHoodieWriteClient(HoodieWriteConfig cfg) { - return getHoodieWriteClient(cfg, false); - } - public HoodieReadClient getHoodieReadClient(String basePath) { readClient = new HoodieReadClient(context, basePath, SQLContext.getOrCreate(jsc.sc())); return readClient; } - public SparkRDDWriteClient getHoodieWriteClient(HoodieWriteConfig cfg, boolean rollbackInflightCommit) { + public SparkRDDWriteClient getHoodieWriteClient(HoodieWriteConfig cfg) { if (null != writeClient) { writeClient.close(); writeClient = null; } - writeClient = new SparkRDDWriteClient(context, cfg, rollbackInflightCommit); + writeClient = new SparkRDDWriteClient(context, cfg); return writeClient; } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieCleaningPolicy.java b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieCleaningPolicy.java index faf22019a53d4..647232fb7e3a9 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieCleaningPolicy.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieCleaningPolicy.java @@ -22,5 +22,5 @@ * Hoodie cleaning policies. */ public enum HoodieCleaningPolicy { - KEEP_LATEST_FILE_VERSIONS, KEEP_LATEST_COMMITS + KEEP_LATEST_FILE_VERSIONS, KEEP_LATEST_COMMITS; } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieFailedWritesCleaningPolicy.java b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieFailedWritesCleaningPolicy.java new file mode 100644 index 0000000000000..f7fef9295a7e7 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieFailedWritesCleaningPolicy.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.common.model; + +/** + * Policy controlling how to perform cleaning for failed writes. + */ +public enum HoodieFailedWritesCleaningPolicy { + // performs cleaning of failed writes inline every write operation + EAGER, + // performs cleaning of failed writes lazily during clean + LAZY, + // Does not clean failed writes + NEVER; + + public boolean isEager() { + return this == EAGER; + } + + public boolean isLazy() { + return this == LAZY; + } + + public boolean isNever() { + return this == NEVER; + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java index 983678db1a6d1..8aa0a3d42cd08 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java @@ -45,6 +45,7 @@ import org.apache.log4j.LogManager; import org.apache.log4j.Logger; +import java.io.File; import java.io.IOException; import java.io.Serializable; import java.util.Arrays; @@ -71,10 +72,10 @@ public class HoodieTableMetaClient implements Serializable { private static final long serialVersionUID = 1L; private static final Logger LOG = LogManager.getLogger(HoodieTableMetaClient.class); public static final String METAFOLDER_NAME = ".hoodie"; - public static final String TEMPFOLDER_NAME = METAFOLDER_NAME + Path.SEPARATOR + ".temp"; - public static final String AUXILIARYFOLDER_NAME = METAFOLDER_NAME + Path.SEPARATOR + ".aux"; - public static final String BOOTSTRAP_INDEX_ROOT_FOLDER_PATH = AUXILIARYFOLDER_NAME + Path.SEPARATOR + ".bootstrap"; - + public static final String TEMPFOLDER_NAME = METAFOLDER_NAME + File.separator + ".temp"; + public static final String AUXILIARYFOLDER_NAME = METAFOLDER_NAME + File.separator + ".aux"; + public static final String BOOTSTRAP_INDEX_ROOT_FOLDER_PATH = AUXILIARYFOLDER_NAME + File.separator + ".bootstrap"; + public static final String HEARTBEAT_FOLDER_NAME = METAFOLDER_NAME + File.separator + ".heartbeat"; public static final String BOOTSTRAP_INDEX_BY_PARTITION_FOLDER_PATH = BOOTSTRAP_INDEX_ROOT_FOLDER_PATH + Path.SEPARATOR + ".partitions"; public static final String BOOTSTRAP_INDEX_BY_FILE_ID_FOLDER_PATH = BOOTSTRAP_INDEX_ROOT_FOLDER_PATH + Path.SEPARATOR @@ -196,6 +197,13 @@ public String getMetaAuxiliaryPath() { return basePath + Path.SEPARATOR + AUXILIARYFOLDER_NAME; } + /** + * @return Heartbeat folder path. + */ + public static String getHeartbeatFolderPath(String basePath) { + return String.format("%s%s%s", basePath, File.separator, HEARTBEAT_FOLDER_NAME); + } + /** * @return Bootstrap Index By Partition Folder */ @@ -516,7 +524,7 @@ public HoodieTimeline getCommitsAndCompactionTimeline() { case COPY_ON_WRITE: return getActiveTimeline().getCommitTimeline(); case MERGE_ON_READ: - return getActiveTimeline().getCommitsAndCompactionTimeline(); + return getActiveTimeline().getWriteTimeline(); default: throw new HoodieException("Unsupported table type :" + this.getTableType()); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/AbstractHoodieLogRecordScanner.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/AbstractHoodieLogRecordScanner.java index 02874e6cb1578..50a44ad0f3f2f 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/AbstractHoodieLogRecordScanner.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/AbstractHoodieLogRecordScanner.java @@ -18,6 +18,11 @@ package org.apache.hudi.common.table.log; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.generic.IndexedRecord; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.model.HoodieRecord; @@ -33,12 +38,6 @@ import org.apache.hudi.common.util.SpillableMapUtils; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieIOException; - -import org.apache.avro.Schema; -import org.apache.avro.generic.GenericRecord; -import org.apache.avro.generic.IndexedRecord; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -53,6 +52,7 @@ import java.util.stream.Collectors; import static org.apache.hudi.common.table.log.block.HoodieLogBlock.HeaderMetadataType.INSTANT_TIME; +import static org.apache.hudi.common.table.log.block.HoodieLogBlock.HoodieLogBlockType.COMMAND_BLOCK; import static org.apache.hudi.common.table.log.block.HoodieLogBlock.HoodieLogBlockType.CORRUPT_BLOCK; /** @@ -125,6 +125,9 @@ public AbstractHoodieLogRecordScanner(FileSystem fs, String basePath, List lastInstantTime = new AtomicReference<>(String.valueOf(Integer.MIN_VALUE)); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieDefaultTimeline.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieDefaultTimeline.java index 484d91b279468..e31f7c19c351f 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieDefaultTimeline.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieDefaultTimeline.java @@ -107,7 +107,7 @@ public HoodieTimeline filterCompletedAndCompactionInstants() { } @Override - public HoodieDefaultTimeline getCommitsAndCompactionTimeline() { + public HoodieDefaultTimeline getWriteTimeline() { Set validActions = CollectionUtils.createSet(COMMIT_ACTION, DELTA_COMMIT_ACTION, COMPACTION_ACTION, REPLACE_COMMIT_ACTION); return new HoodieDefaultTimeline(instants.stream().filter(s -> validActions.contains(s.getAction())), details); } @@ -280,6 +280,11 @@ public boolean containsInstant(HoodieInstant instant) { return instants.stream().anyMatch(s -> s.equals(instant)); } + @Override + public boolean containsInstant(String ts) { + return instants.stream().anyMatch(s -> s.getTimestamp().equals(ts)); + } + @Override public boolean containsOrBeforeTimelineStarts(String instant) { return instants.stream().anyMatch(s -> s.getTimestamp().equals(instant)) || isBeforeTimelineStarts(instant); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieTimeline.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieTimeline.java index ada964fbb9a59..116e9ed0934ae 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieTimeline.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieTimeline.java @@ -131,11 +131,11 @@ public interface HoodieTimeline extends Serializable { HoodieTimeline filterCompletedAndCompactionInstants(); /** - * Timeline to just include commits (commit/deltacommit) and compaction actions. + * Timeline to just include commits (commit/deltacommit), compaction and replace actions. * * @return */ - HoodieTimeline getCommitsAndCompactionTimeline(); + HoodieTimeline getWriteTimeline(); /** * Timeline to just include replace instants that have valid (commit/deltacommit) actions. @@ -156,7 +156,6 @@ public interface HoodieTimeline extends Serializable { */ HoodieTimeline filterPendingReplaceTimeline(); - /** * Create a new Timeline with all the instants after startTs. */ @@ -227,6 +226,11 @@ public interface HoodieTimeline extends Serializable { */ boolean containsInstant(HoodieInstant instant); + /** + * @return true if the passed instant is present as a completed instant on the timeline + */ + boolean containsInstant(String ts); + /** * @return true if the passed instant is present as a completed instant on the timeline or if the instant is before * the first completed instant in the timeline diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/TimelineUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/TimelineUtils.java index 0490e4ec5eede..f9dacf03a5d8d 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/TimelineUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/TimelineUtils.java @@ -49,7 +49,7 @@ public class TimelineUtils { * Does not include internal operations such as clean in the timeline. */ public static List getPartitionsWritten(HoodieTimeline timeline) { - HoodieTimeline timelineToSync = timeline.getCommitsAndCompactionTimeline(); + HoodieTimeline timelineToSync = timeline.getWriteTimeline(); return getAffectedPartitions(timelineToSync); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/AbstractTableFileSystemView.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/AbstractTableFileSystemView.java index 3f457153d081a..f9a60654c721d 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/view/AbstractTableFileSystemView.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/AbstractTableFileSystemView.java @@ -118,7 +118,7 @@ protected void init(HoodieTableMetaClient metaClient, HoodieTimeline visibleActi * @param visibleActiveTimeline Visible Active Timeline */ protected void refreshTimeline(HoodieTimeline visibleActiveTimeline) { - this.visibleCommitsAndCompactionTimeline = visibleActiveTimeline.getCommitsAndCompactionTimeline(); + this.visibleCommitsAndCompactionTimeline = visibleActiveTimeline.getWriteTimeline(); } /** diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/CleanerUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/CleanerUtils.java index 6049ee30752ba..9f63bfa3da4c4 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/CleanerUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/CleanerUtils.java @@ -25,8 +25,10 @@ import org.apache.hudi.avro.model.HoodieCleanerPlan; import org.apache.hudi.common.HoodieCleanStat; import org.apache.hudi.common.model.CleanFileInfo; +import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; import org.apache.hudi.common.table.timeline.versioning.clean.CleanMetadataMigrator; import org.apache.hudi.common.table.timeline.versioning.clean.CleanMetadataV1MigrationHandler; @@ -38,6 +40,8 @@ import java.util.List; import java.util.Map; +import static org.apache.hudi.common.table.timeline.HoodieTimeline.COMMIT_ACTION; + public class CleanerUtils { public static final Integer CLEAN_METADATA_VERSION_1 = CleanMetadataV1MigrationHandler.VERSION; public static final Integer CLEAN_METADATA_VERSION_2 = CleanMetadataV2MigrationHandler.VERSION; @@ -112,4 +116,36 @@ public static HoodieCleanerPlan getCleanerPlan(HoodieTableMetaClient metaClient, public static List convertToHoodieCleanFileInfoList(List cleanFileInfoList) { return cleanFileInfoList.stream().map(CleanFileInfo::toHoodieFileCleanInfo).collect(Collectors.toList()); } + + /** + * Execute {@link HoodieFailedWritesCleaningPolicy} to rollback failed writes for different actions. + * @param cleaningPolicy + * @param actionType + * @param rollbackFailedWritesFunc + */ + public static void rollbackFailedWrites(HoodieFailedWritesCleaningPolicy cleaningPolicy, String actionType, + Functions.Function0 rollbackFailedWritesFunc) { + switch (actionType) { + case HoodieTimeline.CLEAN_ACTION: + if (cleaningPolicy.isEager()) { + // No need to do any special cleanup for failed operations during clean + return; + } else if (cleaningPolicy.isLazy()) { + // Perform rollback of failed operations for all types of actions during clean + rollbackFailedWritesFunc.apply(); + return; + } + // No action needed for cleaning policy NEVER + break; + case COMMIT_ACTION: + // For any other actions, perform rollback of failed writes + if (cleaningPolicy.isEager()) { + rollbackFailedWritesFunc.apply(); + return; + } + break; + default: + throw new IllegalArgumentException("Unsupported action type " + actionType); + } + } } diff --git a/hudi-common/src/main/java/org/apache/hudi/exception/HoodieHeartbeatException.java b/hudi-common/src/main/java/org/apache/hudi/exception/HoodieHeartbeatException.java new file mode 100644 index 0000000000000..eef6baf40e285 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/exception/HoodieHeartbeatException.java @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.exception; + +import java.io.Serializable; + +/** + *

+ * Exception thrown for Hoodie hearbeat failures. The root of the exception hierarchy. + *

+ *

+ * Hoodie Write/Read clients will throw this exception if any of its operations fail. This is a runtime (unchecked) + * exception. + *

+ */ +public class HoodieHeartbeatException extends RuntimeException implements Serializable { + + public HoodieHeartbeatException() { + super(); + } + + public HoodieHeartbeatException(String message) { + super(message); + } + + public HoodieHeartbeatException(String message, Throwable t) { + super(message, t); + } + + public HoodieHeartbeatException(Throwable t) { + super(t); + } + +} diff --git a/hudi-common/src/test/java/org/apache/hudi/common/functional/TestHoodieLogFormat.java b/hudi-common/src/test/java/org/apache/hudi/common/functional/TestHoodieLogFormat.java index 57e814c339090..69fc60f6a3e42 100755 --- a/hudi-common/src/test/java/org/apache/hudi/common/functional/TestHoodieLogFormat.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/functional/TestHoodieLogFormat.java @@ -18,6 +18,13 @@ package org.apache.hudi.common.functional; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.generic.IndexedRecord; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieArchivedLogFile; @@ -39,19 +46,12 @@ import org.apache.hudi.common.table.log.block.HoodieLogBlock; import org.apache.hudi.common.table.log.block.HoodieLogBlock.HeaderMetadataType; import org.apache.hudi.common.table.log.block.HoodieLogBlock.HoodieLogBlockType; +import org.apache.hudi.common.testutils.FileCreateUtils; import org.apache.hudi.common.testutils.HoodieCommonTestHarness; import org.apache.hudi.common.testutils.HoodieTestUtils; import org.apache.hudi.common.testutils.SchemaTestUtil; import org.apache.hudi.common.testutils.minicluster.MiniClusterUtil; import org.apache.hudi.exception.CorruptedLogFileException; - -import org.apache.avro.Schema; -import org.apache.avro.generic.GenericRecord; -import org.apache.avro.generic.IndexedRecord; -import org.apache.hadoop.fs.FSDataOutputStream; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeAll; @@ -472,7 +472,7 @@ public void testBasicAppendAndScanMultipleFiles(boolean readBlocksLazily) writer.appendBlock(dataBlock); } writer.close(); - + FileCreateUtils.createDeltaCommit(basePath, "100", fs); // scan all log blocks (across multiple log files) HoodieMergedLogRecordScanner scanner = HoodieMergedLogRecordScanner.newBuilder() .withFileSystem(fs) @@ -626,6 +626,8 @@ public void testAvroLogRecordReaderBasic(boolean readBlocksLazily) FSUtils.getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION, "100") .map(s -> s.getPath().toString()).collect(Collectors.toList()); + FileCreateUtils.createDeltaCommit(basePath, "100", fs); + HoodieMergedLogRecordScanner scanner = HoodieMergedLogRecordScanner.newBuilder() .withFileSystem(fs) .withBasePath(basePath) @@ -698,6 +700,8 @@ public void testAvroLogRecordReaderWithRollbackTombstone(boolean readBlocksLazil FSUtils.getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION, "100") .map(s -> s.getPath().toString()).collect(Collectors.toList()); + FileCreateUtils.createDeltaCommit(basePath, "100", fs); + FileCreateUtils.createDeltaCommit(basePath, "102", fs); HoodieMergedLogRecordScanner scanner = HoodieMergedLogRecordScanner.newBuilder() .withFileSystem(fs) .withBasePath(basePath) @@ -722,7 +726,7 @@ public void testAvroLogRecordReaderWithRollbackTombstone(boolean readBlocksLazil } @Test - public void testAvroLogRecordReaderWithRollbackPartialBlock() + public void testAvroLogRecordReaderWithFailedPartialBlock() throws IOException, URISyntaxException, InterruptedException { Schema schema = HoodieAvroUtils.addMetadataFields(getSimpleSchema()); // Set a small threshold so that every block is a new version @@ -761,17 +765,9 @@ public void testAvroLogRecordReaderWithRollbackPartialBlock() outputStream.flush(); outputStream.close(); - // Rollback the last write - header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "102"); - header.put(HoodieLogBlock.HeaderMetadataType.TARGET_INSTANT_TIME, "101"); - header.put(HoodieLogBlock.HeaderMetadataType.COMMAND_BLOCK_TYPE, - String.valueOf(HoodieCommandBlock.HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK.ordinal())); - HoodieCommandBlock commandBlock = new HoodieCommandBlock(header); writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath).withFileExtension(HoodieLogFile.DELTA_EXTENSION) .withFileId("test-fileid1").overBaseCommit("100").withFs(fs).build(); - writer.appendBlock(commandBlock); - // Write 3 header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "103"); List records3 = SchemaTestUtil.generateHoodieTestRecords(0, 100); @@ -787,6 +783,8 @@ public void testAvroLogRecordReaderWithRollbackPartialBlock() FSUtils.getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION, "100") .map(s -> s.getPath().toString()).collect(Collectors.toList()); + FileCreateUtils.createDeltaCommit(basePath, "100", fs); + FileCreateUtils.createDeltaCommit(basePath, "103", fs); HoodieMergedLogRecordScanner scanner = HoodieMergedLogRecordScanner.newBuilder() .withFileSystem(fs) .withBasePath(basePath) @@ -857,6 +855,10 @@ public void testAvroLogRecordReaderWithDeleteAndRollback(boolean readBlocksLazil FSUtils.getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION, "100") .map(s -> s.getPath().toString()).collect(Collectors.toList()); + FileCreateUtils.createDeltaCommit(basePath, "100", fs); + FileCreateUtils.createDeltaCommit(basePath, "101", fs); + FileCreateUtils.createDeltaCommit(basePath, "102", fs); + HoodieMergedLogRecordScanner scanner = HoodieMergedLogRecordScanner.newBuilder() .withFileSystem(fs) .withBasePath(basePath) @@ -869,6 +871,7 @@ public void testAvroLogRecordReaderWithDeleteAndRollback(boolean readBlocksLazil .withBufferSize(bufferSize) .withSpillableMapBasePath(BASE_OUTPUT_PATH) .build(); + assertEquals(200, scanner.getTotalLogRecords(), "We still would read 200 records"); final List readKeys = new ArrayList<>(200); final List emptyPayloads = new ArrayList<>(); @@ -897,6 +900,8 @@ public void testAvroLogRecordReaderWithDeleteAndRollback(boolean readBlocksLazil HoodieCommandBlock commandBlock = new HoodieCommandBlock(header); writer.appendBlock(commandBlock); + FileCreateUtils.deleteDeltaCommit(basePath, "102", fs); + readKeys.clear(); scanner = HoodieMergedLogRecordScanner.newBuilder() .withFileSystem(fs) @@ -954,6 +959,8 @@ public void testAvroLogRecordReaderWithFailedRollbacks(boolean readBlocksLazily) HoodieDeleteBlock deleteBlock = new HoodieDeleteBlock(deletedKeys.toArray(new HoodieKey[50]), header); writer.appendBlock(deleteBlock); + FileCreateUtils.createDeltaCommit(basePath, "100", fs); + // Attempt 1 : Write rollback block for a failed write header.put(HoodieLogBlock.HeaderMetadataType.COMMAND_BLOCK_TYPE, String.valueOf(HoodieCommandBlock.HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK.ordinal())); @@ -990,6 +997,7 @@ public void testAvroLogRecordReaderWithFailedRollbacks(boolean readBlocksLazily) final List readKeys = new ArrayList<>(); scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey())); assertEquals(0, readKeys.size(), "Stream collect should return all 0 records"); + FileCreateUtils.deleteDeltaCommit(basePath, "100", fs); } @ParameterizedTest @@ -1023,6 +1031,8 @@ public void testAvroLogRecordReaderWithInsertDeleteAndRollback(boolean readBlock HoodieDeleteBlock deleteBlock = new HoodieDeleteBlock(deletedKeys.toArray(new HoodieKey[50]), header); writer.appendBlock(deleteBlock); + FileCreateUtils.createDeltaCommit(basePath, "100", fs); + // Write 2 rollback blocks (1 data block + 1 delete bloc) for a failed write header.put(HoodieLogBlock.HeaderMetadataType.COMMAND_BLOCK_TYPE, String.valueOf(HoodieCommandBlock.HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK.ordinal())); @@ -1047,6 +1057,7 @@ public void testAvroLogRecordReaderWithInsertDeleteAndRollback(boolean readBlock .withSpillableMapBasePath(BASE_OUTPUT_PATH) .build(); assertEquals(0, scanner.getTotalLogRecords(), "We would read 0 records"); + FileCreateUtils.deleteDeltaCommit(basePath, "100", fs); } @ParameterizedTest @@ -1067,6 +1078,8 @@ public void testAvroLogRecordReaderWithInvalidRollback(boolean readBlocksLazily) HoodieDataBlock dataBlock = getDataBlock(records1, header); writer.appendBlock(dataBlock); + FileCreateUtils.createDeltaCommit(basePath, "100", fs); + // Write invalid rollback for a failed write (possible for in-flight commits) header.put(HoodieLogBlock.HeaderMetadataType.TARGET_INSTANT_TIME, "101"); header.put(HoodieLogBlock.HeaderMetadataType.COMMAND_BLOCK_TYPE, @@ -1130,8 +1143,11 @@ public void testAvroLogRecordReaderWithInsertsDeleteAndRollback(boolean readBloc HoodieDeleteBlock deleteBlock = new HoodieDeleteBlock(deletedKeys.toArray(new HoodieKey[50]), header); writer.appendBlock(deleteBlock); + FileCreateUtils.createDeltaCommit(basePath, "100", fs); + // Write 1 rollback block for a failed write header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "101"); + header.put(HeaderMetadataType.TARGET_INSTANT_TIME, "100"); header.put(HoodieLogBlock.HeaderMetadataType.COMMAND_BLOCK_TYPE, String.valueOf(HoodieCommandBlock.HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK.ordinal())); HoodieCommandBlock commandBlock = new HoodieCommandBlock(header); @@ -1179,6 +1195,8 @@ public void testAvroLogRecordReaderWithMixedInsertsCorruptsAndRollback(boolean r writer.appendBlock(dataBlock); writer.close(); + FileCreateUtils.createDeltaCommit(basePath, "100", fs); + // Append some arbit byte[] to the end of the log (mimics a partially written commit) fs = FSUtils.getFs(fs.getUri().toString(), fs.getConf()); FSDataOutputStream outputStream = fs.append(writer.getLogFile().getPath()); @@ -1254,6 +1272,7 @@ public void testAvroLogRecordReaderWithMixedInsertsCorruptsAndRollback(boolean r .withSpillableMapBasePath(BASE_OUTPUT_PATH) .build(); assertEquals(0, scanner.getTotalLogRecords(), "We would read 0 records"); + FileCreateUtils.deleteDeltaCommit(basePath, "100", fs); } /* @@ -1304,6 +1323,8 @@ private void testAvroLogRecordReaderMergingMultipleLogFiles(int numRecordsInLog1 // Get the size of the block writer2.close(); + FileCreateUtils.createDeltaCommit(basePath, "100", fs); + // From the two log files generated, read the records List allLogFiles = FSUtils.getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION, "100").map(s -> s.getPath().toString()).collect(Collectors.toList()); @@ -1398,6 +1419,8 @@ public void testBasicAppendAndReadInReverse(boolean readBlocksLazily) writer.appendBlock(dataBlock); writer.close(); + FileCreateUtils.createDeltaCommit(basePath, "100", fs); + HoodieLogFileReader reader = new HoodieLogFileReader(fs, writer.getLogFile(), SchemaTestUtil.getSimpleSchema(), bufferSize, readBlocksLazily, true); @@ -1446,6 +1469,8 @@ public void testAppendAndReadOnCorruptedLogInReverse(boolean readBlocksLazily) writer.appendBlock(dataBlock); writer.close(); + FileCreateUtils.createDeltaCommit(basePath, "100", fs); + // Append some arbit byte[] to thee end of the log (mimics a partially written commit) fs = FSUtils.getFs(fs.getUri().toString(), fs.getConf()); FSDataOutputStream outputStream = fs.append(writer.getLogFile().getPath()); @@ -1522,6 +1547,8 @@ public void testBasicAppendAndTraverseInReverse(boolean readBlocksLazily) writer.appendBlock(dataBlock); writer.close(); + FileCreateUtils.createDeltaCommit(basePath, "100", fs); + HoodieLogFileReader reader = new HoodieLogFileReader(fs, writer.getLogFile(), SchemaTestUtil.getSimpleSchema(), bufferSize, readBlocksLazily, true); diff --git a/hudi-common/src/test/java/org/apache/hudi/common/table/timeline/TestHoodieActiveTimeline.java b/hudi-common/src/test/java/org/apache/hudi/common/table/timeline/TestHoodieActiveTimeline.java index 3ed111cc0610a..d80de8ee92147 100755 --- a/hudi-common/src/test/java/org/apache/hudi/common/table/timeline/TestHoodieActiveTimeline.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/table/timeline/TestHoodieActiveTimeline.java @@ -201,7 +201,7 @@ public void testTimelineGetOperations() { // return the correct set of Instant checkTimeline.accept(timeline.getCommitsTimeline(), CollectionUtils.createSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION, HoodieTimeline.REPLACE_COMMIT_ACTION)); - checkTimeline.accept(timeline.getCommitsAndCompactionTimeline(), + checkTimeline.accept(timeline.getWriteTimeline(), CollectionUtils.createSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION, HoodieTimeline.COMPACTION_ACTION, HoodieTimeline.REPLACE_COMMIT_ACTION)); checkTimeline.accept(timeline.getCommitTimeline(), CollectionUtils.createSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.REPLACE_COMMIT_ACTION)); checkTimeline.accept(timeline.getDeltaCommitTimeline(), Collections.singleton(HoodieTimeline.DELTA_COMMIT_ACTION)); diff --git a/hudi-common/src/test/java/org/apache/hudi/common/testutils/FileCreateUtils.java b/hudi-common/src/test/java/org/apache/hudi/common/testutils/FileCreateUtils.java index 2cca148aee7cc..4beba35a58138 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/testutils/FileCreateUtils.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/testutils/FileCreateUtils.java @@ -83,6 +83,17 @@ private static void createMetaFile(String basePath, String instantTime, String s } } + private static void createMetaFile(String basePath, String instantTime, String suffix, FileSystem fs) throws IOException { + org.apache.hadoop.fs.Path parentPath = new org.apache.hadoop.fs.Path(basePath, HoodieTableMetaClient.METAFOLDER_NAME); + if (!fs.exists(parentPath)) { + fs.create(parentPath).close(); + } + org.apache.hadoop.fs.Path metaFilePath = new org.apache.hadoop.fs.Path(parentPath, instantTime + suffix); + if (!fs.exists(metaFilePath)) { + fs.create(metaFilePath).close(); + } + } + private static void createMetaFile(String basePath, String instantTime, String suffix, byte[] content) throws IOException { Path parentPath = Paths.get(basePath, HoodieTableMetaClient.METAFOLDER_NAME); Files.createDirectories(parentPath); @@ -92,10 +103,22 @@ private static void createMetaFile(String basePath, String instantTime, String s } } + private static void deleteMetaFile(String basePath, String instantTime, String suffix, FileSystem fs) throws IOException { + org.apache.hadoop.fs.Path parentPath = new org.apache.hadoop.fs.Path(basePath, HoodieTableMetaClient.METAFOLDER_NAME); + org.apache.hadoop.fs.Path metaFilePath = new org.apache.hadoop.fs.Path(parentPath, instantTime + suffix); + if (fs.exists(metaFilePath)) { + fs.delete(metaFilePath, true); + } + } + public static void createCommit(String basePath, String instantTime) throws IOException { createMetaFile(basePath, instantTime, HoodieTimeline.COMMIT_EXTENSION); } + public static void createCommit(String basePath, String instantTime, FileSystem fs) throws IOException { + createMetaFile(basePath, instantTime, HoodieTimeline.COMMIT_EXTENSION, fs); + } + public static void createRequestedCommit(String basePath, String instantTime) throws IOException { createMetaFile(basePath, instantTime, HoodieTimeline.REQUESTED_COMMIT_EXTENSION); } @@ -108,6 +131,10 @@ public static void createDeltaCommit(String basePath, String instantTime) throws createMetaFile(basePath, instantTime, HoodieTimeline.DELTA_COMMIT_EXTENSION); } + public static void createDeltaCommit(String basePath, String instantTime, FileSystem fs) throws IOException { + createMetaFile(basePath, instantTime, HoodieTimeline.DELTA_COMMIT_EXTENSION, fs); + } + public static void createRequestedDeltaCommit(String basePath, String instantTime) throws IOException { createMetaFile(basePath, instantTime, HoodieTimeline.REQUESTED_DELTA_COMMIT_EXTENSION); } @@ -235,4 +262,8 @@ public static Map getBaseFileCountsForPaths(String basePath, FileS throw new HoodieException("Error reading hoodie table as a dataframe", e); } } + + public static void deleteDeltaCommit(String basePath, String instantTime, FileSystem fs) throws IOException { + deleteMetaFile(basePath, instantTime, HoodieTimeline.DELTA_COMMIT_EXTENSION, fs); + } } diff --git a/hudi-flink/src/main/java/org/apache/hudi/operator/InstantGenerateOperator.java b/hudi-flink/src/main/java/org/apache/hudi/operator/InstantGenerateOperator.java index dea8a056bc4af..8ae8d8524b514 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/operator/InstantGenerateOperator.java +++ b/hudi-flink/src/main/java/org/apache/hudi/operator/InstantGenerateOperator.java @@ -111,7 +111,7 @@ public void open() throws Exception { TaskContextSupplier taskContextSupplier = new FlinkTaskContextSupplier(null); // writeClient - writeClient = new HoodieFlinkWriteClient(new HoodieFlinkEngineContext(taskContextSupplier), StreamerUtil.getHoodieClientConfig(cfg), true); + writeClient = new HoodieFlinkWriteClient(new HoodieFlinkEngineContext(taskContextSupplier), StreamerUtil.getHoodieClientConfig(cfg)); // init table, create it if not exists. StreamerUtil.initTableIfNotExists(FlinkOptions.fromStreamerConfig(cfg)); diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieInputFormatUtils.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieInputFormatUtils.java index eac9f4d6e993b..8585a32adeb64 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieInputFormatUtils.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieInputFormatUtils.java @@ -178,7 +178,7 @@ public static FileInputFormat getInputFormat(String path, boolean realtime, Conf * @return */ public static HoodieDefaultTimeline filterInstantsTimeline(HoodieDefaultTimeline timeline) { - HoodieDefaultTimeline commitsAndCompactionTimeline = timeline.getCommitsAndCompactionTimeline(); + HoodieDefaultTimeline commitsAndCompactionTimeline = timeline.getWriteTimeline(); Option pendingCompactionInstant = commitsAndCompactionTimeline .filterPendingCompactionTimeline().firstInstant(); if (pendingCompactionInstant.isPresent()) { diff --git a/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/realtime/TestHoodieRealtimeRecordReader.java b/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/realtime/TestHoodieRealtimeRecordReader.java index 5a735ce1b5e97..b5571dc37a6fd 100644 --- a/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/realtime/TestHoodieRealtimeRecordReader.java +++ b/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/realtime/TestHoodieRealtimeRecordReader.java @@ -26,6 +26,7 @@ import org.apache.hudi.common.table.log.HoodieLogFormat; import org.apache.hudi.common.table.log.HoodieLogFormat.Writer; import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.testutils.FileCreateUtils; import org.apache.hudi.common.testutils.HoodieTestUtils; import org.apache.hudi.common.testutils.SchemaTestUtil; import org.apache.hudi.common.util.collection.Pair; @@ -126,9 +127,11 @@ private void testReader(boolean partitioned) throws Exception { Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getEvolvedSchema()); HoodieTestUtils.init(hadoopConf, basePath.toString(), HoodieTableType.MERGE_ON_READ); String baseInstant = "100"; - File partitionDir = partitioned ? InputFormatTestUtil.prepareParquetTable(basePath, schema, 1, 100, baseInstant) - : InputFormatTestUtil.prepareNonPartitionedParquetTable(basePath, schema, 1, 100, baseInstant); - InputFormatTestUtil.commit(basePath, baseInstant); + File partitionDir = partitioned ? InputFormatTestUtil.prepareParquetTable(basePath, schema, 1, 100, baseInstant, + HoodieTableType.MERGE_ON_READ) + : InputFormatTestUtil.prepareNonPartitionedParquetTable(basePath, schema, 1, 100, baseInstant, + HoodieTableType.MERGE_ON_READ); + FileCreateUtils.createDeltaCommit(basePath.toString(), baseInstant); // Add the paths FileInputFormat.setInputPaths(jobConf, partitionDir.getPath()); @@ -163,6 +166,7 @@ private void testReader(boolean partitioned) throws Exception { long size = writer.getCurrentSize(); writer.close(); assertTrue(size > 0, "block - size should be > 0"); + FileCreateUtils.createDeltaCommit(basePath.toString(), instantTime); // create a split with baseFile (parquet file written earlier) and new log file(s) fileSlice.addLogFile(writer.getLogFile()); @@ -214,8 +218,9 @@ public void testUnMergedReader() throws Exception { final int numRecords = 1000; final int firstBatchLastRecordKey = numRecords - 1; final int secondBatchLastRecordKey = 2 * numRecords - 1; - File partitionDir = InputFormatTestUtil.prepareParquetTable(basePath, schema, 1, numRecords, instantTime); - InputFormatTestUtil.commit(basePath, instantTime); + File partitionDir = InputFormatTestUtil.prepareParquetTable(basePath, schema, 1, numRecords, instantTime, + HoodieTableType.MERGE_ON_READ); + FileCreateUtils.createDeltaCommit(basePath.toString(), instantTime); // Add the paths FileInputFormat.setInputPaths(jobConf, partitionDir.getPath()); @@ -227,6 +232,7 @@ public void testUnMergedReader() throws Exception { long size = writer.getCurrentSize(); writer.close(); assertTrue(size > 0, "block - size should be > 0"); + FileCreateUtils.createDeltaCommit(basePath.toString(), newCommitTime); // create a split with baseFile (parquet file written earlier) and new log file(s) String logFilePath = writer.getLogFile().getPath().toString(); @@ -291,7 +297,8 @@ public void testReaderWithNestedAndComplexSchema() throws Exception { String instantTime = "100"; int numberOfRecords = 100; int numberOfLogRecords = numberOfRecords / 2; - File partitionDir = InputFormatTestUtil.prepareParquetTable(basePath, schema, 1, numberOfRecords, instantTime); + File partitionDir = InputFormatTestUtil.prepareParquetTable(basePath, schema, 1, numberOfRecords, + instantTime, HoodieTableType.MERGE_ON_READ); InputFormatTestUtil.commit(basePath, instantTime); // Add the paths FileInputFormat.setInputPaths(jobConf, partitionDir.getPath()); @@ -421,7 +428,8 @@ public void testSchemaEvolutionAndRollbackBlockInLastLogFile() throws Exception int numberOfRecords = 100; int numberOfLogRecords = numberOfRecords / 2; File partitionDir = - InputFormatTestUtil.prepareSimpleParquetTable(basePath, schema, 1, numberOfRecords, instantTime); + InputFormatTestUtil.prepareSimpleParquetTable(basePath, schema, 1, numberOfRecords, + instantTime, HoodieTableType.MERGE_ON_READ); InputFormatTestUtil.commit(basePath, instantTime); // Add the paths FileInputFormat.setInputPaths(jobConf, partitionDir.getPath()); diff --git a/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/testutils/InputFormatTestUtil.java b/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/testutils/InputFormatTestUtil.java index b10f38763d6fa..a79d980ac2dbf 100644 --- a/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/testutils/InputFormatTestUtil.java +++ b/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/testutils/InputFormatTestUtil.java @@ -125,7 +125,12 @@ public static void setupIncremental(JobConf jobConf, String startCommit, int num public static File prepareParquetTable(java.nio.file.Path basePath, Schema schema, int numberOfFiles, int numberOfRecords, String commitNumber) throws IOException { - HoodieTestUtils.init(HoodieTestUtils.getDefaultHadoopConf(), basePath.toString()); + return prepareParquetTable(basePath, schema, numberOfFiles, numberOfRecords, commitNumber, HoodieTableType.COPY_ON_WRITE); + } + + public static File prepareParquetTable(java.nio.file.Path basePath, Schema schema, int numberOfFiles, + int numberOfRecords, String commitNumber, HoodieTableType tableType) throws IOException { + HoodieTestUtils.init(HoodieTestUtils.getDefaultHadoopConf(), basePath.toString(), tableType); java.nio.file.Path partitionPath = basePath.resolve(Paths.get("2016", "05", "01")); createData(schema, partitionPath, numberOfFiles, numberOfRecords, commitNumber); return partitionPath.toFile(); @@ -133,7 +138,12 @@ public static File prepareParquetTable(java.nio.file.Path basePath, Schema schem public static File prepareSimpleParquetTable(java.nio.file.Path basePath, Schema schema, int numberOfFiles, int numberOfRecords, String commitNumber) throws Exception { - HoodieTestUtils.init(HoodieTestUtils.getDefaultHadoopConf(), basePath.toString()); + return prepareSimpleParquetTable(basePath, schema, numberOfFiles, numberOfRecords, commitNumber, HoodieTableType.COPY_ON_WRITE); + } + + public static File prepareSimpleParquetTable(java.nio.file.Path basePath, Schema schema, int numberOfFiles, + int numberOfRecords, String commitNumber, HoodieTableType tableType) throws Exception { + HoodieTestUtils.init(HoodieTestUtils.getDefaultHadoopConf(), basePath.toString(), tableType); java.nio.file.Path partitionPath = basePath.resolve(Paths.get("2016", "05", "01")); createSimpleData(schema, partitionPath, numberOfFiles, numberOfRecords, commitNumber); return partitionPath.toFile(); @@ -141,7 +151,12 @@ public static File prepareSimpleParquetTable(java.nio.file.Path basePath, Schema public static File prepareNonPartitionedParquetTable(java.nio.file.Path basePath, Schema schema, int numberOfFiles, int numberOfRecords, String commitNumber) throws IOException { - HoodieTestUtils.init(HoodieTestUtils.getDefaultHadoopConf(), basePath.toString()); + return prepareNonPartitionedParquetTable(basePath, schema, numberOfFiles, numberOfRecords, commitNumber, HoodieTableType.COPY_ON_WRITE); + } + + public static File prepareNonPartitionedParquetTable(java.nio.file.Path basePath, Schema schema, int numberOfFiles, + int numberOfRecords, String commitNumber, HoodieTableType tableType) throws IOException { + HoodieTestUtils.init(HoodieTestUtils.getDefaultHadoopConf(), basePath.toString(), tableType); createData(schema, basePath, numberOfFiles, numberOfRecords, commitNumber); return basePath.toFile(); } diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieTestSuiteWriter.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieTestSuiteWriter.java index 3c6129144582b..c3294dede6d09 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieTestSuiteWriter.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieTestSuiteWriter.java @@ -78,13 +78,7 @@ public class HoodieTestSuiteWriter { Arrays.asList(RollbackNode.class.getName(), CleanNode.class.getName(), ScheduleCompactNode.class.getName())); private static final String GENERATED_DATA_PATH = "generated.data.path"; - public HoodieTestSuiteWriter(JavaSparkContext jsc, Properties props, HoodieTestSuiteConfig cfg, String schema) throws - Exception { - this(jsc, props, cfg, schema, true); - } - - public HoodieTestSuiteWriter(JavaSparkContext jsc, Properties props, HoodieTestSuiteConfig cfg, String schema, - boolean rollbackInflight) throws Exception { + public HoodieTestSuiteWriter(JavaSparkContext jsc, Properties props, HoodieTestSuiteConfig cfg, String schema) throws Exception { // We ensure that only 1 instance of HoodieWriteClient is instantiated for a HoodieTestSuiteWriter // This does not instantiate a HoodieWriteClient until a // {@link HoodieDeltaStreamer#commit(HoodieWriteClient, JavaRDD, Option)} is invoked. @@ -93,7 +87,7 @@ public HoodieTestSuiteWriter(JavaSparkContext jsc, Properties props, HoodieTestS this.hoodieReadClient = new HoodieReadClient(context, cfg.targetBasePath); this.writeConfig = getHoodieClientConfig(cfg, props, schema); if (!cfg.useDeltaStreamer) { - this.writeClient = new SparkRDDWriteClient(context, writeConfig, rollbackInflight); + this.writeClient = new SparkRDDWriteClient(context, writeConfig); } this.cfg = cfg; this.configuration = jsc.hadoopConfiguration(); @@ -234,7 +228,7 @@ public SparkRDDWriteClient getWriteClient(DagNode dagNode) throws IllegalAccessE } synchronized (this) { if (writeClient == null) { - this.writeClient = new SparkRDDWriteClient(new HoodieSparkEngineContext(this.sparkContext), getHoodieClientConfig(cfg, props, schema), false); + this.writeClient = new SparkRDDWriteClient(new HoodieSparkEngineContext(this.sparkContext), getHoodieClientConfig(cfg, props, schema)); } } return writeClient; diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/CompactNode.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/CompactNode.java index 80430661ec4e3..61306d12bcee3 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/CompactNode.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/CompactNode.java @@ -49,7 +49,7 @@ public void execute(ExecutionContext executionContext, int curItrCount) throws E HoodieTableMetaClient.builder().setConf(executionContext.getHoodieTestSuiteWriter().getConfiguration()).setBasePath(executionContext.getHoodieTestSuiteWriter().getCfg().targetBasePath) .build(); Option lastInstant = metaClient.getActiveTimeline() - .getCommitsAndCompactionTimeline().filterPendingCompactionTimeline().lastInstant(); + .getWriteTimeline().filterPendingCompactionTimeline().lastInstant(); if (lastInstant.isPresent()) { log.info("Compacting instant {}", lastInstant.get()); this.result = executionContext.getHoodieTestSuiteWriter().compact(Option.of(lastInstant.get().getTimestamp())); diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/DataSourceUtils.java b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/DataSourceUtils.java index 8d3e81b0b218b..18c51e3fd3eb2 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/DataSourceUtils.java +++ b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/DataSourceUtils.java @@ -190,7 +190,7 @@ public static HoodieWriteConfig createHoodieConfig(String schemaStr, String base public static SparkRDDWriteClient createHoodieClient(JavaSparkContext jssc, String schemaStr, String basePath, String tblName, Map parameters) { - return new SparkRDDWriteClient<>(new HoodieSparkEngineContext(jssc), createHoodieConfig(schemaStr, basePath, tblName, parameters), true); + return new SparkRDDWriteClient<>(new HoodieSparkEngineContext(jssc), createHoodieConfig(schemaStr, basePath, tblName, parameters)); } public static String getCommitActionType(WriteOperationType operation, HoodieTableType tableType) { diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/internal/DataSourceInternalWriterHelper.java b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/internal/DataSourceInternalWriterHelper.java index 7b04a65586393..e45f99a760d9c 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/internal/DataSourceInternalWriterHelper.java +++ b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/internal/DataSourceInternalWriterHelper.java @@ -60,7 +60,7 @@ public DataSourceInternalWriterHelper(String instantTime, HoodieWriteConfig writ SparkSession sparkSession, Configuration configuration) { this.instantTime = instantTime; this.operationType = WriteOperationType.BULK_INSERT; - this.writeClient = new SparkRDDWriteClient<>(new HoodieSparkEngineContext(new JavaSparkContext(sparkSession.sparkContext())), writeConfig, true); + this.writeClient = new SparkRDDWriteClient<>(new HoodieSparkEngineContext(new JavaSparkContext(sparkSession.sparkContext())), writeConfig); writeClient.setOperationType(operationType); writeClient.startCommitWithTime(instantTime); this.metaClient = HoodieTableMetaClient.builder().setConf(configuration).setBasePath(writeConfig.getBasePath()).build(); diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/client/TestBootstrap.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/client/TestBootstrap.java index dbf95de7986d3..6f2858a8c868c 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/client/TestBootstrap.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/client/TestBootstrap.java @@ -18,6 +18,12 @@ package org.apache.hudi.client; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.LongWritable; +import org.apache.hadoop.mapred.JobConf; import org.apache.hudi.DataSourceWriteOptions; import org.apache.hudi.avro.model.HoodieFileStatus; import org.apache.hudi.client.bootstrap.BootstrapMode; @@ -49,13 +55,6 @@ import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.hadoop.HoodieParquetInputFormat; import org.apache.hudi.hadoop.realtime.HoodieParquetRealtimeInputFormat; - -import org.apache.avro.Schema; -import org.apache.avro.generic.GenericRecord; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.mapred.JobConf; -import org.apache.hadoop.io.LongWritable; import org.apache.hudi.index.HoodieIndex.IndexType; import org.apache.hudi.keygen.NonpartitionedKeyGenerator; import org.apache.hudi.keygen.SimpleKeyGenerator; @@ -69,15 +68,14 @@ import org.apache.parquet.schema.MessageType; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.Column; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.SQLContext; -import org.apache.spark.sql.Column; -import org.apache.spark.sql.api.java.UDF1; -import org.apache.spark.sql.types.DataTypes; - import org.apache.spark.sql.SaveMode; import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.api.java.UDF1; +import org.apache.spark.sql.types.DataTypes; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -92,9 +90,9 @@ import java.util.HashSet; import java.util.Iterator; import java.util.List; -import java.util.Set; import java.util.Map; import java.util.Random; +import java.util.Set; import java.util.Spliterators; import java.util.stream.Collectors; import java.util.stream.IntStream; @@ -103,10 +101,10 @@ import static java.util.stream.Collectors.mapping; import static java.util.stream.Collectors.toList; import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.generateGenericRecord; +import static org.apache.spark.sql.functions.callUDF; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertTrue; -import static org.apache.spark.sql.functions.callUDF; /** * Tests Bootstrap Client functionality. @@ -259,7 +257,7 @@ private void testBootstrapCommon(boolean partitioned, boolean deltaCommit, Effec // Rollback Bootstrap FSUtils.deleteInstantFile(metaClient.getFs(), metaClient.getMetaPath(), new HoodieInstant(State.COMPLETED, deltaCommit ? HoodieTimeline.DELTA_COMMIT_ACTION : HoodieTimeline.COMMIT_ACTION, bootstrapCommitInstantTs)); - client.rollBackInflightBootstrap(); + client.rollbackFailedBootstrap(); metaClient.reloadActiveTimeline(); assertEquals(0, metaClient.getCommitsTimeline().countInstants()); assertEquals(0L, BootstrapUtils.getAllLeafFoldersWithFiles(metaClient, metaClient.getFs(), basePath, context) diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieCleaner.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieCleaner.java index 94798eaa44ea7..43101d127fe69 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieCleaner.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieCleaner.java @@ -69,7 +69,7 @@ public HoodieCleaner(Config cfg, JavaSparkContext jssc) { public void run() { HoodieWriteConfig hoodieCfg = getHoodieClientConfig(); - SparkRDDWriteClient client = new SparkRDDWriteClient<>(new HoodieSparkEngineContext(jssc), hoodieCfg, false); + SparkRDDWriteClient client = new SparkRDDWriteClient<>(new HoodieSparkEngineContext(jssc), hoodieCfg); client.clean(); } diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieSnapshotCopier.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieSnapshotCopier.java index 72d1dbdea60b6..6378318b96ded 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieSnapshotCopier.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieSnapshotCopier.java @@ -85,11 +85,11 @@ public void snapshot(JavaSparkContext jsc, String baseDir, final String outputDi final SerializableConfiguration serConf = new SerializableConfiguration(jsc.hadoopConfiguration()); final HoodieTableMetaClient tableMetadata = HoodieTableMetaClient.builder().setConf(fs.getConf()).setBasePath(baseDir).build(); final BaseFileOnlyView fsView = new HoodieTableFileSystemView(tableMetadata, - tableMetadata.getActiveTimeline().getCommitsAndCompactionTimeline().filterCompletedInstants()); + tableMetadata.getActiveTimeline().getWriteTimeline().filterCompletedInstants()); HoodieEngineContext context = new HoodieSparkEngineContext(jsc); // Get the latest commit Option latestCommit = - tableMetadata.getActiveTimeline().getCommitsAndCompactionTimeline().filterCompletedInstants().lastInstant(); + tableMetadata.getActiveTimeline().getWriteTimeline().filterCompletedInstants().lastInstant(); if (!latestCommit.isPresent()) { LOG.warn("No commits present. Nothing to snapshot"); return; diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieSnapshotExporter.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieSnapshotExporter.java index 8e792a0a566c7..ee31d4a600a22 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieSnapshotExporter.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieSnapshotExporter.java @@ -149,7 +149,7 @@ private boolean outputPathExists(FileSystem fs, Config cfg) throws IOException { private Option getLatestCommitTimestamp(FileSystem fs, Config cfg) { final HoodieTableMetaClient tableMetadata = HoodieTableMetaClient.builder().setConf(fs.getConf()).setBasePath(cfg.sourceBasePath).build(); - Option latestCommit = tableMetadata.getActiveTimeline().getCommitsAndCompactionTimeline() + Option latestCommit = tableMetadata.getActiveTimeline().getWriteTimeline() .filterCompletedInstants().lastInstant(); return latestCommit.isPresent() ? Option.of(latestCommit.get().getTimestamp()) : Option.empty(); } @@ -261,7 +261,7 @@ private BaseFileOnlyView getBaseFileOnlyView(JavaSparkContext jsc, Config cfg) { FileSystem fs = FSUtils.getFs(cfg.sourceBasePath, jsc.hadoopConfiguration()); HoodieTableMetaClient tableMetadata = HoodieTableMetaClient.builder().setConf(fs.getConf()).setBasePath(cfg.sourceBasePath).build(); return new HoodieTableFileSystemView(tableMetadata, tableMetadata - .getActiveTimeline().getCommitsAndCompactionTimeline().filterCompletedInstants()); + .getActiveTimeline().getWriteTimeline().filterCompletedInstants()); } public static void main(String[] args) throws IOException { diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/BootstrapExecutor.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/BootstrapExecutor.java index c7974b3eec204..5e34c20dc931c 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/BootstrapExecutor.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/BootstrapExecutor.java @@ -138,7 +138,7 @@ public BootstrapExecutor(HoodieDeltaStreamer.Config cfg, JavaSparkContext jssc, */ public void execute() throws IOException { initializeTable(); - SparkRDDWriteClient bootstrapClient = new SparkRDDWriteClient(new HoodieSparkEngineContext(jssc), bootstrapConfig, true); + SparkRDDWriteClient bootstrapClient = new SparkRDDWriteClient(new HoodieSparkEngineContext(jssc), bootstrapConfig); try { HashMap checkpointCommitMetadata = new HashMap<>(); diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java index 4c494dc2e1ffe..36e2c994f4217 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java @@ -598,7 +598,7 @@ private void reInitWriteClient(Schema sourceSchema, Schema targetSchema) throws // Close Write client. writeClient.close(); } - writeClient = new SparkRDDWriteClient<>(new HoodieSparkEngineContext(jssc), hoodieCfg, true, embeddedTimelineService); + writeClient = new SparkRDDWriteClient<>(new HoodieSparkEngineContext(jssc), hoodieCfg, embeddedTimelineService); onInitializingHoodieWriteClient.apply(writeClient); } diff --git a/pom.xml b/pom.xml index cd66d65141aaf..a80c113a2dad1 100644 --- a/pom.xml +++ b/pom.xml @@ -124,6 +124,7 @@ 1.2.3 1.9.13 1.4.199 + 3.1.2 false ${skipTests} ${skipTests} @@ -969,6 +970,15 @@ 4.0.0 test + + + + org.awaitility + awaitility + ${awaitility.version} + test + +