[HUDI-855] Run Cleaner async with writing (#1577)
- Cleaner can now run concurrently with write operation - Configs to turn on/off Co-authored-by: Vinoth Chandar <vinoth@apache.org>
This commit is contained in:
committed by
GitHub
parent
31247e9b34
commit
8919be6a5d
@@ -0,0 +1,154 @@
|
||||
/*
|
||||
* 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.async;
|
||||
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
import java.io.Serializable;
|
||||
import java.util.concurrent.CompletableFuture;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.Executors;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.function.Function;
|
||||
|
||||
/**
|
||||
* Base Class for running clean/delta-sync/compaction in separate thread and controlling their life-cycle.
|
||||
*/
|
||||
public abstract class AbstractAsyncService implements Serializable {
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(AbstractAsyncService.class);
|
||||
|
||||
// Flag to track if the service is started.
|
||||
private boolean started;
|
||||
// Flag indicating shutdown is externally requested
|
||||
private boolean shutdownRequested;
|
||||
// Flag indicating the service is shutdown
|
||||
private volatile boolean shutdown;
|
||||
// Executor Service for running delta-sync/compaction
|
||||
private transient ExecutorService executor;
|
||||
// Future tracking delta-sync/compaction
|
||||
private transient CompletableFuture future;
|
||||
|
||||
protected AbstractAsyncService() {
|
||||
shutdownRequested = false;
|
||||
}
|
||||
|
||||
protected boolean isShutdownRequested() {
|
||||
return shutdownRequested;
|
||||
}
|
||||
|
||||
protected boolean isShutdown() {
|
||||
return shutdown;
|
||||
}
|
||||
|
||||
/**
|
||||
* Wait till the service shutdown. If the service shutdown with exception, it will be thrown
|
||||
*
|
||||
* @throws ExecutionException
|
||||
* @throws InterruptedException
|
||||
*/
|
||||
public void waitForShutdown() throws ExecutionException, InterruptedException {
|
||||
try {
|
||||
future.get();
|
||||
} catch (ExecutionException ex) {
|
||||
LOG.error("Service shutdown with error", ex);
|
||||
throw ex;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Request shutdown either forcefully or gracefully. Graceful shutdown allows the service to finish up the current
|
||||
* round of work and shutdown. For graceful shutdown, it waits till the service is shutdown
|
||||
*
|
||||
* @param force Forcefully shutdown
|
||||
*/
|
||||
public void shutdown(boolean force) {
|
||||
if (!shutdownRequested || force) {
|
||||
shutdownRequested = true;
|
||||
if (executor != null) {
|
||||
if (force) {
|
||||
executor.shutdownNow();
|
||||
} else {
|
||||
executor.shutdown();
|
||||
try {
|
||||
// Wait for some max time after requesting shutdown
|
||||
executor.awaitTermination(24, TimeUnit.HOURS);
|
||||
} catch (InterruptedException ie) {
|
||||
LOG.error("Interrupted while waiting for shutdown", ie);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Start the service. Runs the service in a different thread and returns. Also starts a monitor thread to
|
||||
* run-callbacks in case of shutdown
|
||||
*
|
||||
* @param onShutdownCallback
|
||||
*/
|
||||
public void start(Function<Boolean, Boolean> onShutdownCallback) {
|
||||
Pair<CompletableFuture, ExecutorService> res = startService();
|
||||
future = res.getKey();
|
||||
executor = res.getValue();
|
||||
started = true;
|
||||
monitorThreads(onShutdownCallback);
|
||||
}
|
||||
|
||||
/**
|
||||
* Service implementation.
|
||||
*
|
||||
* @return
|
||||
*/
|
||||
protected abstract Pair<CompletableFuture, ExecutorService> startService();
|
||||
|
||||
/**
|
||||
* A monitor thread is started which would trigger a callback if the service is shutdown.
|
||||
*
|
||||
* @param onShutdownCallback
|
||||
*/
|
||||
private void monitorThreads(Function<Boolean, Boolean> onShutdownCallback) {
|
||||
LOG.info("Submitting monitor thread !!");
|
||||
Executors.newSingleThreadExecutor().submit(() -> {
|
||||
boolean error = false;
|
||||
try {
|
||||
LOG.info("Monitoring thread(s) !!");
|
||||
future.get();
|
||||
} catch (ExecutionException ex) {
|
||||
LOG.error("Monitor noticed one or more threads failed. Requesting graceful shutdown of other threads", ex);
|
||||
error = true;
|
||||
shutdown(false);
|
||||
} catch (InterruptedException ie) {
|
||||
LOG.error("Got interrupted Monitoring threads", ie);
|
||||
error = true;
|
||||
shutdown(false);
|
||||
} finally {
|
||||
// Mark as shutdown
|
||||
shutdown = true;
|
||||
if (null != onShutdownCallback) {
|
||||
onShutdownCallback.apply(error);
|
||||
}
|
||||
}
|
||||
});
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,85 @@
|
||||
/*
|
||||
* 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;
|
||||
|
||||
import org.apache.hudi.async.AbstractAsyncService;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
import java.util.concurrent.CompletableFuture;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.Executors;
|
||||
|
||||
/**
|
||||
* Clean service running concurrently with write operation.
|
||||
*/
|
||||
class AsyncCleanerService extends AbstractAsyncService {
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(AsyncCleanerService.class);
|
||||
|
||||
private final HoodieWriteClient<?> writeClient;
|
||||
private final String cleanInstantTime;
|
||||
private final transient ExecutorService executor = Executors.newSingleThreadExecutor();
|
||||
|
||||
protected AsyncCleanerService(HoodieWriteClient<?> writeClient, String cleanInstantTime) {
|
||||
this.writeClient = writeClient;
|
||||
this.cleanInstantTime = cleanInstantTime;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Pair<CompletableFuture, ExecutorService> startService() {
|
||||
return Pair.of(CompletableFuture.supplyAsync(() -> {
|
||||
writeClient.clean(cleanInstantTime);
|
||||
return true;
|
||||
}), executor);
|
||||
}
|
||||
|
||||
public static AsyncCleanerService startAsyncCleaningIfEnabled(HoodieWriteClient writeClient,
|
||||
String instantTime) {
|
||||
AsyncCleanerService asyncCleanerService = null;
|
||||
if (writeClient.getConfig().isAutoClean() && writeClient.getConfig().isAsyncClean()) {
|
||||
LOG.info("Auto cleaning is enabled. Running cleaner async to write operation");
|
||||
asyncCleanerService = new AsyncCleanerService(writeClient, instantTime);
|
||||
asyncCleanerService.start(null);
|
||||
} else {
|
||||
LOG.info("Auto cleaning is not enabled. Not running cleaner now");
|
||||
}
|
||||
return asyncCleanerService;
|
||||
}
|
||||
|
||||
public static void waitForCompletion(AsyncCleanerService asyncCleanerService) {
|
||||
if (asyncCleanerService != null) {
|
||||
LOG.info("Waiting for async cleaner to finish");
|
||||
try {
|
||||
asyncCleanerService.waitForShutdown();
|
||||
} catch (Exception e) {
|
||||
throw new HoodieException("Error waiting for async cleaning to finish", e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public static void forceShutdown(AsyncCleanerService asyncCleanerService) {
|
||||
if (asyncCleanerService != null) {
|
||||
LOG.info("Shutting down async cleaner");
|
||||
asyncCleanerService.shutdown(true);
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -80,6 +80,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
|
||||
private final boolean rollbackPending;
|
||||
private final transient HoodieMetrics metrics;
|
||||
private transient Timer.Context compactionTimer;
|
||||
private transient AsyncCleanerService asyncCleanerService;
|
||||
|
||||
/**
|
||||
* Create a write client, without cleaning up failed/inflight commits.
|
||||
@@ -95,28 +96,28 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
|
||||
* Create a write client, with new hudi index.
|
||||
*
|
||||
* @param jsc Java Spark Context
|
||||
* @param clientConfig instance of HoodieWriteConfig
|
||||
* @param writeConfig instance of HoodieWriteConfig
|
||||
* @param rollbackPending whether need to cleanup pending commits
|
||||
*/
|
||||
public HoodieWriteClient(JavaSparkContext jsc, HoodieWriteConfig clientConfig, boolean rollbackPending) {
|
||||
this(jsc, clientConfig, rollbackPending, HoodieIndex.createIndex(clientConfig));
|
||||
public HoodieWriteClient(JavaSparkContext jsc, HoodieWriteConfig writeConfig, boolean rollbackPending) {
|
||||
this(jsc, writeConfig, rollbackPending, HoodieIndex.createIndex(writeConfig));
|
||||
}
|
||||
|
||||
public HoodieWriteClient(JavaSparkContext jsc, HoodieWriteConfig clientConfig, boolean rollbackPending, HoodieIndex index) {
|
||||
this(jsc, clientConfig, rollbackPending, index, Option.empty());
|
||||
public HoodieWriteClient(JavaSparkContext jsc, HoodieWriteConfig writeConfig, boolean rollbackPending, HoodieIndex index) {
|
||||
this(jsc, writeConfig, rollbackPending, index, Option.empty());
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a write client, allows to specify all parameters.
|
||||
*
|
||||
* @param jsc Java Spark Context
|
||||
* @param clientConfig instance of HoodieWriteConfig
|
||||
* @param writeConfig instance of HoodieWriteConfig
|
||||
* @param rollbackPending whether need to cleanup pending commits
|
||||
* @param timelineService Timeline Service that runs as part of write client.
|
||||
*/
|
||||
public HoodieWriteClient(JavaSparkContext jsc, HoodieWriteConfig clientConfig, boolean rollbackPending,
|
||||
public HoodieWriteClient(JavaSparkContext jsc, HoodieWriteConfig writeConfig, boolean rollbackPending,
|
||||
HoodieIndex index, Option<EmbeddedTimelineService> timelineService) {
|
||||
super(jsc, index, clientConfig, timelineService);
|
||||
super(jsc, index, writeConfig, timelineService);
|
||||
this.metrics = new HoodieMetrics(config, config.getTableName());
|
||||
this.rollbackPending = rollbackPending;
|
||||
}
|
||||
@@ -158,6 +159,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
|
||||
HoodieTable<T> table = getTableAndInitCtx(WriteOperationType.UPSERT);
|
||||
table.validateUpsertSchema();
|
||||
setOperationType(WriteOperationType.UPSERT);
|
||||
this.asyncCleanerService = AsyncCleanerService.startAsyncCleaningIfEnabled(this, instantTime);
|
||||
HoodieWriteMetadata result = table.upsert(jsc, instantTime, records);
|
||||
if (result.getIndexLookupDuration().isPresent()) {
|
||||
metrics.updateIndexMetrics(LOOKUP_STR, result.getIndexLookupDuration().get().toMillis());
|
||||
@@ -178,6 +180,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
|
||||
HoodieTable<T> table = getTableAndInitCtx(WriteOperationType.UPSERT_PREPPED);
|
||||
table.validateUpsertSchema();
|
||||
setOperationType(WriteOperationType.UPSERT_PREPPED);
|
||||
this.asyncCleanerService = AsyncCleanerService.startAsyncCleaningIfEnabled(this, instantTime);
|
||||
HoodieWriteMetadata result = table.upsertPrepped(jsc,instantTime, preppedRecords);
|
||||
return postWrite(result, instantTime, table);
|
||||
}
|
||||
@@ -196,6 +199,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
|
||||
HoodieTable<T> table = getTableAndInitCtx(WriteOperationType.INSERT);
|
||||
table.validateInsertSchema();
|
||||
setOperationType(WriteOperationType.INSERT);
|
||||
this.asyncCleanerService = AsyncCleanerService.startAsyncCleaningIfEnabled(this, instantTime);
|
||||
HoodieWriteMetadata result = table.insert(jsc,instantTime, records);
|
||||
return postWrite(result, instantTime, table);
|
||||
}
|
||||
@@ -215,6 +219,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
|
||||
HoodieTable<T> table = getTableAndInitCtx(WriteOperationType.INSERT_PREPPED);
|
||||
table.validateInsertSchema();
|
||||
setOperationType(WriteOperationType.INSERT_PREPPED);
|
||||
this.asyncCleanerService = AsyncCleanerService.startAsyncCleaningIfEnabled(this, instantTime);
|
||||
HoodieWriteMetadata result = table.insertPrepped(jsc,instantTime, preppedRecords);
|
||||
return postWrite(result, instantTime, table);
|
||||
}
|
||||
@@ -254,6 +259,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
|
||||
HoodieTable<T> table = getTableAndInitCtx(WriteOperationType.BULK_INSERT);
|
||||
table.validateInsertSchema();
|
||||
setOperationType(WriteOperationType.BULK_INSERT);
|
||||
this.asyncCleanerService = AsyncCleanerService.startAsyncCleaningIfEnabled(this, instantTime);
|
||||
HoodieWriteMetadata result = table.bulkInsert(jsc,instantTime, records, bulkInsertPartitioner);
|
||||
return postWrite(result, instantTime, table);
|
||||
}
|
||||
@@ -279,6 +285,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
|
||||
HoodieTable<T> table = getTableAndInitCtx(WriteOperationType.BULK_INSERT_PREPPED);
|
||||
table.validateInsertSchema();
|
||||
setOperationType(WriteOperationType.BULK_INSERT_PREPPED);
|
||||
this.asyncCleanerService = AsyncCleanerService.startAsyncCleaningIfEnabled(this, instantTime);
|
||||
HoodieWriteMetadata result = table.bulkInsertPrepped(jsc,instantTime, preppedRecords, bulkInsertPartitioner);
|
||||
return postWrite(result, instantTime, table);
|
||||
}
|
||||
@@ -338,18 +345,30 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
|
||||
// We cannot have unbounded commit files. Archive commits if we have to archive
|
||||
HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(config, createMetaClient(true));
|
||||
archiveLog.archiveIfRequired(hadoopConf);
|
||||
if (config.isAutoClean()) {
|
||||
// Call clean to cleanup if there is anything to cleanup after the commit,
|
||||
LOG.info("Auto cleaning is enabled. Running cleaner now");
|
||||
clean(instantTime);
|
||||
} else {
|
||||
LOG.info("Auto cleaning is not enabled. Not running cleaner now");
|
||||
}
|
||||
autoCleanOnCommit(instantTime);
|
||||
} catch (IOException ioe) {
|
||||
throw new HoodieIOException(ioe.getMessage(), ioe);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Handle auto clean during commit.
|
||||
* @param instantTime
|
||||
*/
|
||||
private void autoCleanOnCommit(String instantTime) {
|
||||
if (config.isAutoClean()) {
|
||||
// Call clean to cleanup if there is anything to cleanup after the commit,
|
||||
if (config.isAsyncClean()) {
|
||||
LOG.info("Cleaner has been spawned already. Waiting for it to finish");
|
||||
AsyncCleanerService.waitForCompletion(asyncCleanerService);
|
||||
LOG.info("Cleaner has finished");
|
||||
} else {
|
||||
LOG.info("Auto cleaning is enabled. Running cleaner now");
|
||||
clean(instantTime);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a savepoint based on the latest commit action on the timeline.
|
||||
*
|
||||
@@ -477,7 +496,8 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
|
||||
*/
|
||||
@Override
|
||||
public void close() {
|
||||
// Stop timeline-server if running
|
||||
AsyncCleanerService.forceShutdown(asyncCleanerService);
|
||||
asyncCleanerService = null;
|
||||
super.close();
|
||||
}
|
||||
|
||||
|
||||
@@ -40,6 +40,8 @@ public class HoodieCompactionConfig extends DefaultHoodieConfig {
|
||||
|
||||
public static final String CLEANER_POLICY_PROP = "hoodie.cleaner.policy";
|
||||
public static final String AUTO_CLEAN_PROP = "hoodie.clean.automatic";
|
||||
public static final String ASYNC_CLEAN_PROP = "hoodie.clean.async";
|
||||
|
||||
// Turn on inline compaction - after fw delta commits a inline compaction will be run
|
||||
public static final String INLINE_COMPACT_PROP = "hoodie.compact.inline";
|
||||
// Run a compaction every N delta commits
|
||||
@@ -101,6 +103,7 @@ public class HoodieCompactionConfig extends DefaultHoodieConfig {
|
||||
public static final String DEFAULT_COMPACTION_REVERSE_LOG_READ_ENABLED = "false";
|
||||
private static final String DEFAULT_CLEANER_POLICY = HoodieCleaningPolicy.KEEP_LATEST_COMMITS.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";
|
||||
private static final String DEFAULT_INCREMENTAL_CLEANER = "true";
|
||||
private static final String DEFAULT_INLINE_COMPACT_NUM_DELTA_COMMITS = "5";
|
||||
@@ -143,6 +146,11 @@ public class HoodieCompactionConfig extends DefaultHoodieConfig {
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withAsyncClean(Boolean asyncClean) {
|
||||
props.setProperty(ASYNC_CLEAN_PROP, String.valueOf(asyncClean));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withIncrementalCleaningMode(Boolean incrementalCleaningMode) {
|
||||
props.setProperty(CLEANER_INCREMENTAL_MODE, String.valueOf(incrementalCleaningMode));
|
||||
return this;
|
||||
@@ -247,6 +255,8 @@ public class HoodieCompactionConfig extends DefaultHoodieConfig {
|
||||
public HoodieCompactionConfig build() {
|
||||
HoodieCompactionConfig config = new HoodieCompactionConfig(props);
|
||||
setDefaultOnCondition(props, !props.containsKey(AUTO_CLEAN_PROP), AUTO_CLEAN_PROP, DEFAULT_AUTO_CLEAN);
|
||||
setDefaultOnCondition(props, !props.containsKey(ASYNC_CLEAN_PROP), ASYNC_CLEAN_PROP,
|
||||
DEFAULT_ASYNC_CLEAN);
|
||||
setDefaultOnCondition(props, !props.containsKey(CLEANER_INCREMENTAL_MODE), CLEANER_INCREMENTAL_MODE,
|
||||
DEFAULT_INCREMENTAL_CLEANER);
|
||||
setDefaultOnCondition(props, !props.containsKey(INLINE_COMPACT_PROP), INLINE_COMPACT_PROP,
|
||||
|
||||
@@ -296,6 +296,10 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
|
||||
return Boolean.parseBoolean(props.getProperty(HoodieCompactionConfig.AUTO_CLEAN_PROP));
|
||||
}
|
||||
|
||||
public boolean isAsyncClean() {
|
||||
return Boolean.parseBoolean(props.getProperty(HoodieCompactionConfig.ASYNC_CLEAN_PROP));
|
||||
}
|
||||
|
||||
public boolean incrementalCleanerModeEnabled() {
|
||||
return Boolean.parseBoolean(props.getProperty(HoodieCompactionConfig.CLEANER_INCREMENTAL_MODE));
|
||||
}
|
||||
|
||||
@@ -185,8 +185,9 @@ public class CleanActionExecutor extends BaseActionExecutor<HoodieCleanMetadata>
|
||||
Option<HoodieCleanerPlan> requestClean(String startCleanTime) {
|
||||
final HoodieCleanerPlan cleanerPlan = requestClean(jsc);
|
||||
if ((cleanerPlan.getFilesToBeDeletedPerPartition() != null)
|
||||
&& !cleanerPlan.getFilesToBeDeletedPerPartition().isEmpty()) {
|
||||
|
||||
&& !cleanerPlan.getFilesToBeDeletedPerPartition().isEmpty()
|
||||
&& cleanerPlan.getFilesToBeDeletedPerPartition().values().stream().mapToInt(List::size).sum() > 0) {
|
||||
// Only create cleaner plan which does some work
|
||||
final HoodieInstant cleanInstant = new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.CLEAN_ACTION, startCleanTime);
|
||||
// Save to both aux and timeline folder
|
||||
try {
|
||||
|
||||
@@ -51,6 +51,7 @@ import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.config.HoodieCompactionConfig;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.index.HoodieIndex;
|
||||
import org.apache.hudi.testutils.HoodieClientTestBase;
|
||||
import org.apache.hudi.testutils.HoodieTestDataGenerator;
|
||||
@@ -83,6 +84,7 @@ import scala.Tuple3;
|
||||
import static org.apache.hudi.common.testutils.HoodieTestUtils.DEFAULT_PARTITION_PATHS;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertFalse;
|
||||
import static org.junit.jupiter.api.Assertions.assertNull;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
|
||||
/**
|
||||
@@ -128,15 +130,8 @@ public class TestCleaner extends HoodieClientTestBase {
|
||||
HoodieTable table = HoodieTable.create(metaClient, client.getConfig(), hadoopConf);
|
||||
|
||||
assertFalse(table.getCompletedCommitsTimeline().empty());
|
||||
if (cleaningPolicy.equals(HoodieCleaningPolicy.KEEP_LATEST_COMMITS)) {
|
||||
// We no longer write empty cleaner plans when there are not enough commits present
|
||||
assertTrue(table.getCompletedCleanTimeline().empty());
|
||||
} else {
|
||||
String instantTime = table.getCompletedCommitsTimeline().getInstants().findFirst().get().getTimestamp();
|
||||
assertFalse(table.getCompletedCleanTimeline().empty());
|
||||
assertEquals(instantTime, table.getCompletedCleanTimeline().getInstants().findFirst().get().getTimestamp(),
|
||||
"The clean instant should be the same as the commit instant");
|
||||
}
|
||||
// We no longer write empty cleaner plans when there is nothing to be cleaned.
|
||||
assertTrue(table.getCompletedCleanTimeline().empty());
|
||||
|
||||
HoodieIndex index = HoodieIndex.createIndex(cfg);
|
||||
List<HoodieRecord> taggedRecords = index.tagLocation(jsc.parallelize(records, 1), jsc, table).collect();
|
||||
@@ -439,17 +434,32 @@ public class TestCleaner extends HoodieClientTestBase {
|
||||
|
||||
if (simulateRetryFailure) {
|
||||
HoodieInstant completedCleanInstant = new HoodieInstant(State.COMPLETED, HoodieTimeline.CLEAN_ACTION, cleanInstantTs);
|
||||
HoodieCleanMetadata metadata = CleanerUtils.getCleanerMetadata(metaClient, completedCleanInstant);
|
||||
metadata.getPartitionMetadata().values().forEach(p -> {
|
||||
String dirPath = metaClient.getBasePath() + "/" + p.getPartitionPath();
|
||||
p.getSuccessDeleteFiles().forEach(p2 -> {
|
||||
try {
|
||||
metaClient.getFs().create(new Path(dirPath, p2), true);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException(e.getMessage(), e);
|
||||
}
|
||||
});
|
||||
});
|
||||
metaClient.reloadActiveTimeline().revertToInflight(completedCleanInstant);
|
||||
HoodieCleanMetadata cleanMetadata2 = writeClient.clean(getNextInstant());
|
||||
HoodieCleanMetadata newCleanMetadata = writeClient.clean(getNextInstant());
|
||||
// No new clean metadata would be created. Only the previous one will be retried
|
||||
assertNull(newCleanMetadata);
|
||||
HoodieCleanMetadata cleanMetadata2 = CleanerUtils.getCleanerMetadata(metaClient, completedCleanInstant);
|
||||
assertEquals(cleanMetadata1.getEarliestCommitToRetain(), cleanMetadata2.getEarliestCommitToRetain());
|
||||
assertEquals(new Integer(0), cleanMetadata2.getTotalFilesDeleted());
|
||||
assertEquals(cleanMetadata1.getTotalFilesDeleted(), cleanMetadata2.getTotalFilesDeleted());
|
||||
assertEquals(cleanMetadata1.getPartitionMetadata().keySet(), cleanMetadata2.getPartitionMetadata().keySet());
|
||||
final HoodieCleanMetadata retriedCleanMetadata = CleanerUtils.getCleanerMetadata(HoodieTableMetaClient.reload(metaClient), completedCleanInstant);
|
||||
cleanMetadata1.getPartitionMetadata().keySet().forEach(k -> {
|
||||
HoodieCleanPartitionMetadata p1 = cleanMetadata1.getPartitionMetadata().get(k);
|
||||
HoodieCleanPartitionMetadata p2 = retriedCleanMetadata.getPartitionMetadata().get(k);
|
||||
assertEquals(p1.getDeletePathPatterns(), p2.getDeletePathPatterns());
|
||||
assertEquals(p1.getSuccessDeleteFiles(), p2.getFailedDeleteFiles());
|
||||
assertEquals(p1.getSuccessDeleteFiles(), p2.getSuccessDeleteFiles());
|
||||
assertEquals(p1.getFailedDeleteFiles(), p2.getFailedDeleteFiles());
|
||||
assertEquals(p1.getPartitionPath(), p2.getPartitionPath());
|
||||
assertEquals(k, p1.getPartitionPath());
|
||||
});
|
||||
@@ -487,12 +497,7 @@ public class TestCleaner extends HoodieClientTestBase {
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
|
||||
List<HoodieCleanStat> hoodieCleanStatsOne = runCleaner(config);
|
||||
assertEquals(0,
|
||||
getCleanStat(hoodieCleanStatsOne, HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH).getSuccessDeleteFiles()
|
||||
.size(), "Must not clean any files");
|
||||
assertEquals(0,
|
||||
getCleanStat(hoodieCleanStatsOne, HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH).getSuccessDeleteFiles()
|
||||
.size(), "Must not clean any files");
|
||||
assertEquals(0, hoodieCleanStatsOne.size(), "Must not clean any files");
|
||||
assertTrue(HoodieTestUtils.doesDataFileExist(basePath, HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, "000",
|
||||
file1P0C0));
|
||||
assertTrue(HoodieTestUtils.doesDataFileExist(basePath, HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH, "000",
|
||||
@@ -548,9 +553,7 @@ public class TestCleaner extends HoodieClientTestBase {
|
||||
// No cleaning on partially written file, with no commit.
|
||||
HoodieTestUtils.createDataFile(basePath, HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, "003", file3P0C2); // update
|
||||
List<HoodieCleanStat> hoodieCleanStatsFour = runCleaner(config);
|
||||
assertEquals(0,
|
||||
getCleanStat(hoodieCleanStatsFour, HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH).getSuccessDeleteFiles()
|
||||
.size(), "Must not clean any files");
|
||||
assertEquals(0, hoodieCleanStatsFour.size(), "Must not clean any files");
|
||||
assertTrue(HoodieTestUtils.doesDataFileExist(basePath, HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, "002",
|
||||
file3P0C2));
|
||||
}
|
||||
@@ -819,11 +822,8 @@ public class TestCleaner extends HoodieClientTestBase {
|
||||
Option.of(commitMetadata.toJsonString().getBytes(StandardCharsets.UTF_8)));
|
||||
|
||||
List<HoodieCleanStat> hoodieCleanStatsThree = runCleaner(config, simulateFailureRetry);
|
||||
assertEquals(0,
|
||||
getCleanStat(hoodieCleanStatsThree, HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH)
|
||||
.getSuccessDeleteFiles().size(),
|
||||
assertEquals(0, hoodieCleanStatsThree.size(),
|
||||
"Must not clean any file. We have to keep 1 version before the latest commit time to keep");
|
||||
|
||||
assertTrue(HoodieTestUtils.doesDataFileExist(basePath, HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, "000",
|
||||
file1P0C0));
|
||||
|
||||
|
||||
Reference in New Issue
Block a user