[HUDI-137] Hudi cleaning state changes should be consistent with compaction actions
Before this change, Cleaner performs cleaning of old file versions and then stores the deleted files in .clean files. With this setup, we will not be able to track file deletions if a cleaner fails after deleting files but before writing .clean metadata. This is fine for regular file-system view generation but Incremental timeline syncing relies on clean/commit/compaction metadata to keep a consistent file-system view. Cleaner state transitions is now similar to that of compaction. 1. Requested : HoodieWriteClient.scheduleClean() selects the list of files that needs to be deleted and stores them in metadata 2. Inflight : HoodieWriteClient marks the state to be inflight before it starts deleting 3. Completed : HoodieWriteClient marks the state after completing the deletion according to the cleaner plan
This commit is contained in:
committed by
Balaji Varadarajan
parent
23b303e4b1
commit
1032fc3e54
@@ -288,8 +288,7 @@ public class CompactionCommand implements CommandMarker {
|
||||
String message = "\n\n\t COMPACTION PLAN " + (valid ? "VALID" : "INVALID") + "\n\n";
|
||||
List<Comparable[]> rows = new ArrayList<>();
|
||||
res.stream().forEach(r -> {
|
||||
Comparable[] row = new Comparable[]{r.getOperation().getFileId(),
|
||||
r.getOperation().getBaseInstantTime(),
|
||||
Comparable[] row = new Comparable[] {r.getOperation().getFileId(), r.getOperation().getBaseInstantTime(),
|
||||
r.getOperation().getDataFileName().isPresent() ? r.getOperation().getDataFileName().get() : "",
|
||||
r.getOperation().getDeltaFileNames().size(), r.isSuccess(),
|
||||
r.getException().isPresent() ? r.getException().get().getMessage() : ""};
|
||||
|
||||
@@ -239,8 +239,7 @@ public class CompactionAdminClient extends AbstractHoodieClient {
|
||||
FileSlice merged =
|
||||
fileSystemView.getLatestMergedFileSlicesBeforeOrOn(op.getPartitionPath(), lastInstant.getTimestamp())
|
||||
.filter(fs -> fs.getFileId().equals(op.getFileId())).findFirst().get();
|
||||
final int maxVersion =
|
||||
op.getDeltaFileNames().stream().map(lf -> FSUtils.getFileVersionFromLog(new Path(lf)))
|
||||
final int maxVersion = op.getDeltaFileNames().stream().map(lf -> FSUtils.getFileVersionFromLog(new Path(lf)))
|
||||
.reduce((x, y) -> x > y ? x : y).orElse(0);
|
||||
List<HoodieLogFile> logFilesToBeMoved =
|
||||
merged.getLogFiles().filter(lf -> lf.getLogVersion() > maxVersion).collect(Collectors.toList());
|
||||
@@ -293,22 +292,21 @@ public class CompactionAdminClient extends AbstractHoodieClient {
|
||||
FileSlice fs = fileSliceOptional.get();
|
||||
Option<HoodieDataFile> df = fs.getDataFile();
|
||||
if (operation.getDataFileName().isPresent()) {
|
||||
String expPath = metaClient.getFs().getFileStatus(new Path(
|
||||
FSUtils.getPartitionPath(metaClient.getBasePath(), operation.getPartitionPath()),
|
||||
new Path(operation.getDataFileName().get()))).getPath()
|
||||
.toString();
|
||||
Preconditions.checkArgument(df.isPresent(), "Data File must be present. File Slice was : "
|
||||
+ fs + ", operation :" + operation);
|
||||
String expPath = metaClient.getFs()
|
||||
.getFileStatus(
|
||||
new Path(FSUtils.getPartitionPath(metaClient.getBasePath(), operation.getPartitionPath()),
|
||||
new Path(operation.getDataFileName().get())))
|
||||
.getPath().toString();
|
||||
Preconditions.checkArgument(df.isPresent(),
|
||||
"Data File must be present. File Slice was : " + fs + ", operation :" + operation);
|
||||
Preconditions.checkArgument(df.get().getPath().equals(expPath),
|
||||
"Base Path in operation is specified as " + expPath + " but got path " + df.get().getPath());
|
||||
}
|
||||
Set<HoodieLogFile> logFilesInFileSlice = fs.getLogFiles().collect(Collectors.toSet());
|
||||
Set<HoodieLogFile> logFilesInCompactionOp = operation.getDeltaFileNames().stream()
|
||||
.map(dp -> {
|
||||
Set<HoodieLogFile> logFilesInCompactionOp = operation.getDeltaFileNames().stream().map(dp -> {
|
||||
try {
|
||||
FileStatus[] fileStatuses = metaClient.getFs().listStatus(
|
||||
new Path(FSUtils.getPartitionPath(metaClient.getBasePath(), operation.getPartitionPath()),
|
||||
new Path(dp)));
|
||||
FileStatus[] fileStatuses = metaClient.getFs().listStatus(new Path(
|
||||
FSUtils.getPartitionPath(metaClient.getBasePath(), operation.getPartitionPath()), new Path(dp)));
|
||||
Preconditions.checkArgument(fileStatuses.length == 1, "Expect only 1 file-status");
|
||||
return new HoodieLogFile(fileStatuses[0]);
|
||||
} catch (FileNotFoundException fe) {
|
||||
@@ -317,8 +315,7 @@ public class CompactionAdminClient extends AbstractHoodieClient {
|
||||
throw new HoodieIOException(ioe.getMessage(), ioe);
|
||||
}
|
||||
}).collect(Collectors.toSet());
|
||||
Set<HoodieLogFile> missing =
|
||||
logFilesInCompactionOp.stream().filter(lf -> !logFilesInFileSlice.contains(lf))
|
||||
Set<HoodieLogFile> missing = logFilesInCompactionOp.stream().filter(lf -> !logFilesInFileSlice.contains(lf))
|
||||
.collect(Collectors.toSet());
|
||||
Preconditions.checkArgument(missing.isEmpty(),
|
||||
"All log files specified in compaction operation is not present. Missing :" + missing + ", Exp :"
|
||||
|
||||
182
hudi-client/src/main/java/org/apache/hudi/HoodieCleanClient.java
Normal file
182
hudi-client/src/main/java/org/apache/hudi/HoodieCleanClient.java
Normal file
@@ -0,0 +1,182 @@
|
||||
/*
|
||||
* 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;
|
||||
|
||||
import com.codahale.metrics.Timer;
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.base.Preconditions;
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import org.apache.hudi.avro.model.HoodieCleanMetadata;
|
||||
import org.apache.hudi.avro.model.HoodieCleanerPlan;
|
||||
import org.apache.hudi.client.embedded.EmbeddedTimelineService;
|
||||
import org.apache.hudi.common.HoodieCleanStat;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.table.HoodieTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant.State;
|
||||
import org.apache.hudi.common.util.AvroUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.metrics.HoodieMetrics;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
|
||||
public class HoodieCleanClient<T extends HoodieRecordPayload> extends AbstractHoodieClient {
|
||||
|
||||
private static Logger logger = LogManager.getLogger(HoodieCleanClient.class);
|
||||
private final transient HoodieMetrics metrics;
|
||||
|
||||
public HoodieCleanClient(JavaSparkContext jsc, HoodieWriteConfig clientConfig, HoodieMetrics metrics) {
|
||||
this(jsc, clientConfig, metrics, Option.empty());
|
||||
}
|
||||
|
||||
public HoodieCleanClient(JavaSparkContext jsc, HoodieWriteConfig clientConfig, HoodieMetrics metrics,
|
||||
Option<EmbeddedTimelineService> timelineService) {
|
||||
super(jsc, clientConfig, timelineService);
|
||||
this.metrics = metrics;
|
||||
}
|
||||
|
||||
/**
|
||||
* Clean up any stale/old files/data lying around (either on file storage or index storage) based on the
|
||||
* configurations and CleaningPolicy used. (typically files that no longer can be used by a running query can be
|
||||
* cleaned)
|
||||
*/
|
||||
public void clean() throws HoodieIOException {
|
||||
String startCleanTime = HoodieActiveTimeline.createNewCommitTime();
|
||||
clean(startCleanTime);
|
||||
}
|
||||
|
||||
/**
|
||||
* Clean up any stale/old files/data lying around (either on file storage or index storage) based on the
|
||||
* configurations and CleaningPolicy used. (typically files that no longer can be used by a running query can be
|
||||
* cleaned)
|
||||
*
|
||||
* @param startCleanTime Cleaner Instant Timestamp
|
||||
* @throws HoodieIOException in case of any IOException
|
||||
*/
|
||||
protected HoodieCleanMetadata clean(String startCleanTime) throws HoodieIOException {
|
||||
// Create a Hoodie table which encapsulated the commits and files visible
|
||||
final HoodieTable<T> table = HoodieTable.getHoodieTable(createMetaClient(true), config, jsc);
|
||||
|
||||
// If there are inflight(failed) or previously requested clean operation, first perform them
|
||||
table.getCleanTimeline().filterInflightsAndRequested().getInstants().forEach(hoodieInstant -> {
|
||||
logger.info("There were previously unfinished cleaner operations. Finishing Instant=" + hoodieInstant);
|
||||
runClean(table, hoodieInstant.getTimestamp());
|
||||
});
|
||||
|
||||
Option<HoodieCleanerPlan> cleanerPlanOpt = scheduleClean(startCleanTime);
|
||||
|
||||
if (cleanerPlanOpt.isPresent()) {
|
||||
HoodieCleanerPlan cleanerPlan = cleanerPlanOpt.get();
|
||||
if ((cleanerPlan.getFilesToBeDeletedPerPartition() != null)
|
||||
&& !cleanerPlan.getFilesToBeDeletedPerPartition().isEmpty()) {
|
||||
final HoodieTable<T> hoodieTable = HoodieTable.getHoodieTable(createMetaClient(true), config, jsc);
|
||||
return runClean(hoodieTable, startCleanTime);
|
||||
}
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a Cleaner plan if there are files to be cleaned and stores them in instant file
|
||||
*
|
||||
* @param startCleanTime Cleaner Instant Time
|
||||
* @return Cleaner Plan if generated
|
||||
*/
|
||||
@VisibleForTesting
|
||||
protected Option<HoodieCleanerPlan> scheduleClean(String startCleanTime) {
|
||||
// Create a Hoodie table which encapsulated the commits and files visible
|
||||
HoodieTable<T> table = HoodieTable.getHoodieTable(createMetaClient(true), config, jsc);
|
||||
|
||||
HoodieCleanerPlan cleanerPlan = table.scheduleClean(jsc);
|
||||
|
||||
if ((cleanerPlan.getFilesToBeDeletedPerPartition() != null)
|
||||
&& !cleanerPlan.getFilesToBeDeletedPerPartition().isEmpty()) {
|
||||
|
||||
HoodieInstant cleanInstant = new HoodieInstant(State.REQUESTED, HoodieTimeline.CLEAN_ACTION, startCleanTime);
|
||||
// Save to both aux and timeline folder
|
||||
try {
|
||||
table.getActiveTimeline().saveToCleanRequested(cleanInstant, AvroUtils.serializeCleanerPlan(cleanerPlan));
|
||||
logger.info("Requesting Cleaning with instant time " + cleanInstant);
|
||||
} catch (IOException e) {
|
||||
logger.error("Got exception when saving cleaner requested file", e);
|
||||
throw new HoodieIOException(e.getMessage(), e);
|
||||
}
|
||||
return Option.of(cleanerPlan);
|
||||
}
|
||||
return Option.empty();
|
||||
}
|
||||
|
||||
/**
|
||||
* Executes the Cleaner plan stored in the instant metadata
|
||||
*
|
||||
* @param table Hoodie Table
|
||||
* @param cleanInstantTs Cleaner Instant Timestamp
|
||||
*/
|
||||
@VisibleForTesting
|
||||
protected HoodieCleanMetadata runClean(HoodieTable<T> table, String cleanInstantTs) {
|
||||
HoodieInstant cleanInstant =
|
||||
table.getCleanTimeline().getInstants().filter(x -> x.getTimestamp().equals(cleanInstantTs)).findFirst().get();
|
||||
|
||||
Preconditions.checkArgument(
|
||||
cleanInstant.getState().equals(State.REQUESTED) || cleanInstant.getState().equals(State.INFLIGHT));
|
||||
|
||||
try {
|
||||
logger.info("Cleaner started");
|
||||
final Timer.Context context = metrics.getCleanCtx();
|
||||
|
||||
if (!cleanInstant.isInflight()) {
|
||||
// Mark as inflight first
|
||||
cleanInstant = table.getActiveTimeline().transitionCleanRequestedToInflight(cleanInstant);
|
||||
}
|
||||
|
||||
List<HoodieCleanStat> cleanStats = table.clean(jsc, cleanInstant);
|
||||
|
||||
if (cleanStats.isEmpty()) {
|
||||
return HoodieCleanMetadata.newBuilder().build();
|
||||
}
|
||||
|
||||
// Emit metrics (duration, numFilesDeleted) if needed
|
||||
Option<Long> durationInMs = Option.empty();
|
||||
if (context != null) {
|
||||
durationInMs = Option.of(metrics.getDurationInMs(context.stop()));
|
||||
logger.info("cleanerElaspsedTime (Minutes): " + durationInMs.get() / (1000 * 60));
|
||||
}
|
||||
|
||||
// Create the metadata and save it
|
||||
HoodieCleanMetadata metadata =
|
||||
AvroUtils.convertCleanMetadata(cleanInstant.getTimestamp(), durationInMs, cleanStats);
|
||||
logger.info("Cleaned " + metadata.getTotalFilesDeleted() + " files");
|
||||
metrics.updateCleanMetrics(durationInMs.orElseGet(() -> -1L), metadata.getTotalFilesDeleted());
|
||||
|
||||
table.getActiveTimeline().transitionCleanInflightToComplete(
|
||||
new HoodieInstant(true, HoodieTimeline.CLEAN_ACTION, cleanInstant.getTimestamp()),
|
||||
AvroUtils.serializeCleanMetadata(metadata));
|
||||
logger.info("Marked clean started on " + cleanInstant.getTimestamp() + " as complete");
|
||||
return metadata;
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Failed to clean up after commit", e);
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -38,7 +38,6 @@ import org.apache.hudi.avro.model.HoodieRestoreMetadata;
|
||||
import org.apache.hudi.avro.model.HoodieRollbackMetadata;
|
||||
import org.apache.hudi.avro.model.HoodieSavepointMetadata;
|
||||
import org.apache.hudi.client.embedded.EmbeddedTimelineService;
|
||||
import org.apache.hudi.common.HoodieCleanStat;
|
||||
import org.apache.hudi.common.HoodieRollbackStat;
|
||||
import org.apache.hudi.common.model.HoodieCommitMetadata;
|
||||
import org.apache.hudi.common.model.HoodieDataFile;
|
||||
@@ -98,6 +97,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
|
||||
private final boolean rollbackInFlight;
|
||||
private final transient HoodieMetrics metrics;
|
||||
private final transient HoodieIndex<T> index;
|
||||
private final transient HoodieCleanClient<T> cleanClient;
|
||||
private transient Timer.Context writeContext = null;
|
||||
private transient Timer.Context compactionTimer;
|
||||
private transient Timer.Context indexTimer = null;
|
||||
@@ -131,6 +131,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
|
||||
this.index = index;
|
||||
this.metrics = new HoodieMetrics(config, config.getTableName());
|
||||
this.rollbackInFlight = rollbackInFlight;
|
||||
this.cleanClient = new HoodieCleanClient<>(jsc, config, metrics, timelineService);
|
||||
}
|
||||
|
||||
public static SparkConf registerClasses(SparkConf conf) {
|
||||
@@ -918,6 +919,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
|
||||
public void close() {
|
||||
// Stop timeline-server if running
|
||||
super.close();
|
||||
this.cleanClient.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();
|
||||
@@ -927,55 +929,24 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
|
||||
* Clean up any stale/old files/data lying around (either on file storage or index storage) based on the
|
||||
* configurations and CleaningPolicy used. (typically files that no longer can be used by a running query can be
|
||||
* cleaned)
|
||||
*
|
||||
* @throws HoodieIOException
|
||||
*/
|
||||
public void clean() throws HoodieIOException {
|
||||
String startCleanTime = HoodieActiveTimeline.createNewCommitTime();
|
||||
clean(startCleanTime);
|
||||
cleanClient.clean();
|
||||
}
|
||||
|
||||
/**
|
||||
* Clean up any stale/old files/data lying around (either on file storage or index storage) based on the
|
||||
* configurations and CleaningPolicy used. (typically files that no longer can be used by a running query can be
|
||||
* cleaned)
|
||||
*
|
||||
* @param startCleanTime Cleaner Instant Timestamp
|
||||
* @return
|
||||
* @throws HoodieIOException in case of any IOException
|
||||
*/
|
||||
private void clean(String startCleanTime) throws HoodieIOException {
|
||||
try {
|
||||
logger.info("Cleaner started");
|
||||
final Timer.Context context = metrics.getCleanCtx();
|
||||
|
||||
// Create a Hoodie table which encapsulated the commits and files visible
|
||||
HoodieTable<T> table = HoodieTable.getHoodieTable(createMetaClient(true), config, jsc);
|
||||
|
||||
List<HoodieCleanStat> cleanStats = table.clean(jsc);
|
||||
if (cleanStats.isEmpty()) {
|
||||
return;
|
||||
}
|
||||
|
||||
// Emit metrics (duration, numFilesDeleted) if needed
|
||||
Option<Long> durationInMs = Option.empty();
|
||||
if (context != null) {
|
||||
durationInMs = Option.of(metrics.getDurationInMs(context.stop()));
|
||||
logger.info("cleanerElaspsedTime (Minutes): " + durationInMs.get() / (1000 * 60));
|
||||
}
|
||||
|
||||
// Create the metadata and save it
|
||||
HoodieCleanMetadata metadata = AvroUtils.convertCleanMetadata(startCleanTime, durationInMs, cleanStats);
|
||||
logger.info("Cleaned " + metadata.getTotalFilesDeleted() + " files");
|
||||
metrics.updateCleanMetrics(durationInMs.orElseGet(() -> -1L), metadata.getTotalFilesDeleted());
|
||||
|
||||
table.getActiveTimeline().saveAsComplete(new HoodieInstant(true, HoodieTimeline.CLEAN_ACTION, startCleanTime),
|
||||
AvroUtils.serializeCleanMetadata(metadata));
|
||||
logger.info("Marked clean started on " + startCleanTime + " as complete");
|
||||
|
||||
if (!table.getActiveTimeline().getCleanerTimeline().empty()) {
|
||||
// Cleanup of older cleaner meta files
|
||||
// TODO - make the commit archival generic and archive clean metadata
|
||||
FSUtils.deleteOlderCleanMetaFiles(fs, table.getMetaClient().getMetaPath(),
|
||||
table.getActiveTimeline().getCleanerTimeline().getInstants());
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Failed to clean up after commit", e);
|
||||
}
|
||||
protected HoodieCleanMetadata clean(String startCleanTime) throws HoodieIOException {
|
||||
return cleanClient.clean(startCleanTime);
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -1176,8 +1147,8 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
|
||||
private JavaRDD<WriteStatus> runCompaction(HoodieInstant compactionInstant, HoodieActiveTimeline activeTimeline,
|
||||
boolean autoCommit) throws IOException {
|
||||
HoodieTableMetaClient metaClient = createMetaClient(true);
|
||||
HoodieCompactionPlan compactionPlan = CompactionUtils.getCompactionPlan(metaClient,
|
||||
compactionInstant.getTimestamp());
|
||||
HoodieCompactionPlan compactionPlan =
|
||||
CompactionUtils.getCompactionPlan(metaClient, compactionInstant.getTimestamp());
|
||||
|
||||
// Mark instant as compaction inflight
|
||||
activeTimeline.transitionCompactionRequestedToInflight(compactionInstant);
|
||||
|
||||
@@ -97,9 +97,11 @@ public class EmbeddedTimelineService {
|
||||
|
||||
public void stop() {
|
||||
if (null != server) {
|
||||
logger.info("Closing Timeline server");
|
||||
this.server.close();
|
||||
this.server = null;
|
||||
this.viewManager = null;
|
||||
logger.info("Closed Timeline server");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -19,6 +19,7 @@
|
||||
package org.apache.hudi.io;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
@@ -52,7 +53,7 @@ import org.apache.log4j.Logger;
|
||||
* <p>
|
||||
* TODO: Should all cleaning be done based on {@link HoodieCommitMetadata}
|
||||
*/
|
||||
public class HoodieCleanHelper<T extends HoodieRecordPayload<T>> {
|
||||
public class HoodieCleanHelper<T extends HoodieRecordPayload<T>> implements Serializable {
|
||||
|
||||
private static Logger logger = LogManager.getLogger(HoodieCleanHelper.class);
|
||||
|
||||
@@ -114,12 +115,11 @@ public class HoodieCleanHelper<T extends HoodieRecordPayload<T>> {
|
||||
FileSlice nextSlice = fileSliceIterator.next();
|
||||
if (nextSlice.getDataFile().isPresent()) {
|
||||
HoodieDataFile dataFile = nextSlice.getDataFile().get();
|
||||
deletePaths.add(dataFile.getPath());
|
||||
deletePaths.add(dataFile.getFileName());
|
||||
}
|
||||
if (hoodieTable.getMetaClient().getTableType() == HoodieTableType.MERGE_ON_READ) {
|
||||
// If merge on read, then clean the log files for the commits as well
|
||||
deletePaths
|
||||
.addAll(nextSlice.getLogFiles().map(file -> file.getPath().toString()).collect(Collectors.toList()));
|
||||
deletePaths.addAll(nextSlice.getLogFiles().map(file -> file.getFileName()).collect(Collectors.toList()));
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -187,11 +187,10 @@ public class HoodieCleanHelper<T extends HoodieRecordPayload<T>> {
|
||||
if (!isFileSliceNeededForPendingCompaction(aSlice) && HoodieTimeline
|
||||
.compareTimestamps(earliestCommitToRetain.getTimestamp(), fileCommitTime, HoodieTimeline.GREATER)) {
|
||||
// this is a commit, that should be cleaned.
|
||||
aFile.ifPresent(hoodieDataFile -> deletePaths.add(hoodieDataFile.getPath()));
|
||||
aFile.ifPresent(hoodieDataFile -> deletePaths.add(hoodieDataFile.getFileName()));
|
||||
if (hoodieTable.getMetaClient().getTableType() == HoodieTableType.MERGE_ON_READ) {
|
||||
// If merge on read, then clean the log files for the commits as well
|
||||
deletePaths
|
||||
.addAll(aSlice.getLogFiles().map(file -> file.getPath().toString()).collect(Collectors.toList()));
|
||||
deletePaths.addAll(aSlice.getLogFiles().map(file -> file.getFileName()).collect(Collectors.toList()));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -195,9 +195,8 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieWrit
|
||||
// Load the new records in a map
|
||||
long memoryForMerge = config.getMaxMemoryPerPartitionMerge();
|
||||
logger.info("MaxMemoryPerPartitionMerge => " + memoryForMerge);
|
||||
this.keyToNewRecords = new ExternalSpillableMap<>(memoryForMerge,
|
||||
config.getSpillableMapBasePath(), new DefaultSizeEstimator(),
|
||||
new HoodieRecordSizeEstimator(originalSchema));
|
||||
this.keyToNewRecords = new ExternalSpillableMap<>(memoryForMerge, config.getSpillableMapBasePath(),
|
||||
new DefaultSizeEstimator(), new HoodieRecordSizeEstimator(originalSchema));
|
||||
} catch (IOException io) {
|
||||
throw new HoodieIOException("Cannot instantiate an ExternalSpillableMap", io);
|
||||
}
|
||||
|
||||
@@ -101,8 +101,8 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
|
||||
FileSystem fs = metaClient.getFs();
|
||||
|
||||
Schema readerSchema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(config.getSchema()));
|
||||
log.info("Compacting base " + operation.getDataFileName() + " with delta files " + operation
|
||||
.getDeltaFileNames() + " for commit " + commitTime);
|
||||
log.info("Compacting base " + operation.getDataFileName() + " with delta files " + operation.getDeltaFileNames()
|
||||
+ " for commit " + commitTime);
|
||||
// TODO - FIX THIS
|
||||
// Reads the entire avro file. Always only specific blocks should be read from the avro file
|
||||
// (failure recover).
|
||||
@@ -115,20 +115,19 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
|
||||
.filterCompletedInstants().lastInstant().get().getTimestamp();
|
||||
log.info("MaxMemoryPerCompaction => " + config.getMaxMemoryPerCompaction());
|
||||
|
||||
List<String> logFiles = operation.getDeltaFileNames().stream()
|
||||
.map(p -> new Path(FSUtils.getPartitionPath(metaClient.getBasePath(), operation.getPartitionPath()),
|
||||
p).toString()).collect(toList());
|
||||
HoodieMergedLogRecordScanner scanner = new HoodieMergedLogRecordScanner(fs,
|
||||
metaClient.getBasePath(), logFiles, readerSchema, maxInstantTime,
|
||||
config.getMaxMemoryPerCompaction(), config.getCompactionLazyBlockReadEnabled(),
|
||||
List<String> logFiles = operation.getDeltaFileNames().stream().map(
|
||||
p -> new Path(FSUtils.getPartitionPath(metaClient.getBasePath(), operation.getPartitionPath()), p).toString())
|
||||
.collect(toList());
|
||||
HoodieMergedLogRecordScanner scanner = new HoodieMergedLogRecordScanner(fs, metaClient.getBasePath(), logFiles,
|
||||
readerSchema, maxInstantTime, config.getMaxMemoryPerCompaction(), config.getCompactionLazyBlockReadEnabled(),
|
||||
config.getCompactionReverseLogReadEnabled(), config.getMaxDFSStreamBufferSize(),
|
||||
config.getSpillableMapBasePath());
|
||||
if (!scanner.iterator().hasNext()) {
|
||||
return Lists.<WriteStatus>newArrayList();
|
||||
}
|
||||
|
||||
Option<HoodieDataFile> oldDataFileOpt = operation.getBaseFile(metaClient.getBasePath(),
|
||||
operation.getPartitionPath());
|
||||
Option<HoodieDataFile> oldDataFileOpt =
|
||||
operation.getBaseFile(metaClient.getBasePath(), operation.getPartitionPath());
|
||||
|
||||
// Compacting is very similar to applying updates to existing file
|
||||
Iterator<List<WriteStatus>> result;
|
||||
@@ -189,16 +188,12 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
|
||||
|
||||
RealtimeView fileSystemView = hoodieTable.getRTFileSystemView();
|
||||
log.info("Compaction looking for files to compact in " + partitionPaths + " partitions");
|
||||
List<HoodieCompactionOperation> operations =
|
||||
jsc.parallelize(partitionPaths, partitionPaths.size())
|
||||
List<HoodieCompactionOperation> operations = jsc.parallelize(partitionPaths, partitionPaths.size())
|
||||
.flatMap((FlatMapFunction<String, CompactionOperation>) partitionPath -> fileSystemView
|
||||
.getLatestFileSlices(partitionPath)
|
||||
.filter(slice ->
|
||||
!fgIdsInPendingCompactions.contains(slice.getFileGroupId()))
|
||||
.map(
|
||||
s -> {
|
||||
List<HoodieLogFile> logFiles = s.getLogFiles().sorted(HoodieLogFile
|
||||
.getLogFileComparator()).collect(Collectors.toList());
|
||||
.filter(slice -> !fgIdsInPendingCompactions.contains(slice.getFileGroupId())).map(s -> {
|
||||
List<HoodieLogFile> logFiles =
|
||||
s.getLogFiles().sorted(HoodieLogFile.getLogFileComparator()).collect(Collectors.toList());
|
||||
totalLogFiles.add((long) logFiles.size());
|
||||
totalFileSlices.add(1L);
|
||||
// Avro generated classes are not inheriting Serializable. Using CompactionOperation POJO
|
||||
@@ -207,10 +202,8 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
|
||||
Option<HoodieDataFile> dataFile = s.getDataFile();
|
||||
return new CompactionOperation(dataFile, partitionPath, logFiles,
|
||||
config.getCompactionStrategy().captureMetrics(config, dataFile, partitionPath, logFiles));
|
||||
})
|
||||
.filter(c -> !c.getDeltaFileNames().isEmpty())
|
||||
.collect(toList()).iterator()).collect().stream().map(CompactionUtils::buildHoodieCompactionOperation)
|
||||
.collect(toList());
|
||||
}).filter(c -> !c.getDeltaFileNames().isEmpty()).collect(toList()).iterator())
|
||||
.collect().stream().map(CompactionUtils::buildHoodieCompactionOperation).collect(toList());
|
||||
log.info("Total of " + operations.size() + " compactions are retrieved");
|
||||
log.info("Total number of latest files slices " + totalFileSlices.value());
|
||||
log.info("Total number of log files " + totalLogFiles.value());
|
||||
|
||||
@@ -96,8 +96,7 @@ public abstract class CompactionStrategy implements Serializable {
|
||||
// Strategy implementation can overload this method to set specific compactor-id
|
||||
return HoodieCompactionPlan.newBuilder()
|
||||
.setOperations(orderAndFilter(writeConfig, operations, pendingCompactionPlans))
|
||||
.setVersion(CompactionUtils.LATEST_COMPACTION_METADATA_VERSION)
|
||||
.build();
|
||||
.setVersion(CompactionUtils.LATEST_COMPACTION_METADATA_VERSION).build();
|
||||
}
|
||||
|
||||
/**
|
||||
|
||||
@@ -19,6 +19,7 @@
|
||||
package org.apache.hudi.table;
|
||||
|
||||
import com.google.common.hash.Hashing;
|
||||
import java.io.FileNotFoundException;
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
@@ -35,9 +36,12 @@ import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.WriteStatus;
|
||||
import org.apache.hudi.avro.model.HoodieActionInstant;
|
||||
import org.apache.hudi.avro.model.HoodieCleanerPlan;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionPlan;
|
||||
import org.apache.hudi.common.HoodieCleanStat;
|
||||
import org.apache.hudi.common.HoodieRollbackStat;
|
||||
import org.apache.hudi.common.model.HoodieCleaningPolicy;
|
||||
import org.apache.hudi.common.model.HoodieCommitMetadata;
|
||||
import org.apache.hudi.common.model.HoodieDataFile;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
@@ -48,6 +52,7 @@ import org.apache.hudi.common.model.HoodieRollingStatMetadata;
|
||||
import org.apache.hudi.common.table.HoodieTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.util.AvroUtils;
|
||||
import org.apache.hudi.common.util.FSUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
@@ -72,10 +77,10 @@ import org.apache.parquet.hadoop.ParquetReader;
|
||||
import org.apache.spark.Partitioner;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.api.java.function.Function2;
|
||||
import org.apache.spark.api.java.function.PairFlatMapFunction;
|
||||
import scala.Tuple2;
|
||||
|
||||
|
||||
/**
|
||||
* Implementation of a very heavily read-optimized Hoodie Table where
|
||||
* <p>
|
||||
@@ -97,10 +102,12 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
|
||||
Map<String, PartitionCleanStat> partitionCleanStatMap = new HashMap<>();
|
||||
|
||||
FileSystem fs = table.getMetaClient().getFs();
|
||||
Path basePath = new Path(table.getMetaClient().getBasePath());
|
||||
while (iter.hasNext()) {
|
||||
Tuple2<String, String> partitionDelFileTuple = iter.next();
|
||||
String partitionPath = partitionDelFileTuple._1();
|
||||
String deletePathStr = partitionDelFileTuple._2();
|
||||
String delFileName = partitionDelFileTuple._2();
|
||||
String deletePathStr = new Path(new Path(basePath, partitionPath), delFileName).toString();
|
||||
Boolean deletedFileResult = deleteFileAndGetResult(fs, deletePathStr);
|
||||
if (!partitionCleanStatMap.containsKey(partitionPath)) {
|
||||
partitionCleanStatMap.put(partitionPath, new PartitionCleanStat(partitionPath));
|
||||
@@ -109,29 +116,24 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
|
||||
partitionCleanStat.addDeleteFilePatterns(deletePathStr);
|
||||
partitionCleanStat.addDeletedFileResult(deletePathStr, deletedFileResult);
|
||||
}
|
||||
|
||||
return partitionCleanStatMap.entrySet().stream().map(e -> new Tuple2<>(e.getKey(), e.getValue()))
|
||||
.collect(Collectors.toList()).iterator();
|
||||
};
|
||||
}
|
||||
|
||||
private static PairFlatMapFunction<String, String, String> getFilesToDeleteFunc(HoodieTable table,
|
||||
HoodieWriteConfig config) {
|
||||
return (PairFlatMapFunction<String, String, String>) partitionPathToClean -> {
|
||||
HoodieCleanHelper cleaner = new HoodieCleanHelper(table, config);
|
||||
return cleaner.getDeletePaths(partitionPathToClean).stream()
|
||||
.map(deleteFile -> new Tuple2<>(partitionPathToClean, deleteFile.toString())).iterator();
|
||||
};
|
||||
}
|
||||
|
||||
private static Boolean deleteFileAndGetResult(FileSystem fs, String deletePathStr) throws IOException {
|
||||
Path deletePath = new Path(deletePathStr);
|
||||
logger.debug("Working on delete path :" + deletePath);
|
||||
try {
|
||||
boolean deleteResult = fs.delete(deletePath, false);
|
||||
if (deleteResult) {
|
||||
logger.debug("Cleaned file at path :" + deletePath);
|
||||
}
|
||||
return deleteResult;
|
||||
} catch (FileNotFoundException fio) {
|
||||
// With cleanPlan being used for retried cleaning operations, its possible to clean a file twice
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
@@ -268,6 +270,40 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
|
||||
return handleUpsertPartition(commitTime, partition, recordItr, partitioner);
|
||||
}
|
||||
|
||||
/**
|
||||
* Generates List of files to be cleaned
|
||||
*
|
||||
* @param jsc JavaSparkContext
|
||||
* @return Cleaner Plan
|
||||
*/
|
||||
public HoodieCleanerPlan scheduleClean(JavaSparkContext jsc) {
|
||||
try {
|
||||
FileSystem fs = getMetaClient().getFs();
|
||||
|
||||
List<String> partitionsToClean =
|
||||
FSUtils.getAllPartitionPaths(fs, getMetaClient().getBasePath(), config.shouldAssumeDatePartitioning());
|
||||
if (partitionsToClean.isEmpty()) {
|
||||
logger.info("Nothing to clean here. It is already clean");
|
||||
return HoodieCleanerPlan.newBuilder().setPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS.name()).build();
|
||||
}
|
||||
logger.info(
|
||||
"Total Partitions to clean : " + partitionsToClean.size() + ", with policy " + config.getCleanerPolicy());
|
||||
int cleanerParallelism = Math.min(partitionsToClean.size(), config.getCleanerParallelism());
|
||||
logger.info("Using cleanerParallelism: " + cleanerParallelism);
|
||||
HoodieCleanHelper cleaner = new HoodieCleanHelper(this, config);
|
||||
Option<HoodieInstant> earliestInstant = cleaner.getEarliestCommitToRetain();
|
||||
|
||||
Map<String, List<String>> cleanOps = jsc.parallelize(partitionsToClean, cleanerParallelism)
|
||||
.map(partitionPathToClean -> Pair.of(partitionPathToClean, cleaner.getDeletePaths(partitionPathToClean)))
|
||||
.collect().stream().collect(Collectors.toMap(Pair::getKey, Pair::getValue));
|
||||
return new HoodieCleanerPlan(earliestInstant
|
||||
.map(x -> new HoodieActionInstant(x.getTimestamp(), x.getAction(), x.getState().name())).orElse(null),
|
||||
config.getCleanerPolicy().name(), cleanOps, 1);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Failed to schedule clean operation", e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Performs cleaning of partition paths according to cleaning policy and returns the number of files cleaned. Handles
|
||||
* skews in partitions to clean by making files to clean as the unit of task distribution.
|
||||
@@ -275,17 +311,40 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
|
||||
* @throws IllegalArgumentException if unknown cleaning policy is provided
|
||||
*/
|
||||
@Override
|
||||
public List<HoodieCleanStat> clean(JavaSparkContext jsc) {
|
||||
public List<HoodieCleanStat> clean(JavaSparkContext jsc, HoodieInstant cleanInstant) {
|
||||
try {
|
||||
FileSystem fs = getMetaClient().getFs();
|
||||
List<String> partitionsToClean =
|
||||
FSUtils.getAllPartitionPaths(fs, getMetaClient().getBasePath(), config.shouldAssumeDatePartitioning());
|
||||
logger.info("Partitions to clean up : " + partitionsToClean + ", with policy " + config.getCleanerPolicy());
|
||||
if (partitionsToClean.isEmpty()) {
|
||||
logger.info("Nothing to clean here mom. It is already clean");
|
||||
return Collections.emptyList();
|
||||
}
|
||||
return cleanPartitionPaths(partitionsToClean, jsc);
|
||||
HoodieCleanerPlan cleanerPlan = AvroUtils.deserializeCleanerPlan(getActiveTimeline()
|
||||
.getInstantAuxiliaryDetails(HoodieTimeline.getCleanRequestedInstant(cleanInstant.getTimestamp())).get());
|
||||
|
||||
int cleanerParallelism = Math.min(
|
||||
(int) (cleanerPlan.getFilesToBeDeletedPerPartition().values().stream().mapToInt(x -> x.size()).count()),
|
||||
config.getCleanerParallelism());
|
||||
logger.info("Using cleanerParallelism: " + cleanerParallelism);
|
||||
List<Tuple2<String, PartitionCleanStat>> partitionCleanStats = jsc
|
||||
.parallelize(cleanerPlan.getFilesToBeDeletedPerPartition().entrySet().stream()
|
||||
.flatMap(x -> x.getValue().stream().map(y -> new Tuple2<String, String>(x.getKey(), y)))
|
||||
.collect(Collectors.toList()), cleanerParallelism)
|
||||
.mapPartitionsToPair(deleteFilesFunc(this)).reduceByKey((e1, e2) -> e1.merge(e2)).collect();
|
||||
|
||||
Map<String, PartitionCleanStat> partitionCleanStatsMap =
|
||||
partitionCleanStats.stream().collect(Collectors.toMap(Tuple2::_1, Tuple2::_2));
|
||||
|
||||
// Return PartitionCleanStat for each partition passed.
|
||||
return cleanerPlan.getFilesToBeDeletedPerPartition().keySet().stream().map(partitionPath -> {
|
||||
PartitionCleanStat partitionCleanStat =
|
||||
(partitionCleanStatsMap.containsKey(partitionPath)) ? partitionCleanStatsMap.get(partitionPath)
|
||||
: new PartitionCleanStat(partitionPath);
|
||||
HoodieActionInstant actionInstant = cleanerPlan.getEarliestInstantToRetain();
|
||||
return HoodieCleanStat.newBuilder().withPolicy(config.getCleanerPolicy()).withPartitionPath(partitionPath)
|
||||
.withEarliestCommitRetained(Option.ofNullable(
|
||||
actionInstant != null
|
||||
? new HoodieInstant(HoodieInstant.State.valueOf(actionInstant.getState()),
|
||||
actionInstant.getAction(), actionInstant.getTimestamp())
|
||||
: null))
|
||||
.withDeletePathPattern(partitionCleanStat.deletePathPatterns)
|
||||
.withSuccessfulDeletes(partitionCleanStat.successDeleteFiles)
|
||||
.withFailedDeletes(partitionCleanStat.failedDeleteFiles).build();
|
||||
}).collect(Collectors.toList());
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Failed to clean up after commit", e);
|
||||
}
|
||||
@@ -311,7 +370,7 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
|
||||
logger.info("Clean out all parquet files generated for commit: " + commit);
|
||||
List<RollbackRequest> rollbackRequests = generateRollbackRequests(instantToRollback);
|
||||
|
||||
//TODO: We need to persist this as rollback workload and use it in case of partial failures
|
||||
// TODO: We need to persist this as rollback workload and use it in case of partial failures
|
||||
List<HoodieRollbackStat> stats =
|
||||
new RollbackExecutor(metaClient, config).performRollback(jsc, instantToRollback, rollbackRequests);
|
||||
|
||||
@@ -321,8 +380,7 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
|
||||
return stats;
|
||||
}
|
||||
|
||||
private List<RollbackRequest> generateRollbackRequests(HoodieInstant instantToRollback)
|
||||
throws IOException {
|
||||
private List<RollbackRequest> generateRollbackRequests(HoodieInstant instantToRollback) throws IOException {
|
||||
return FSUtils.getAllPartitionPaths(this.metaClient.getFs(), this.getMetaClient().getBasePath(),
|
||||
config.shouldAssumeDatePartitioning()).stream().map(partitionPath -> {
|
||||
return RollbackRequest.createRollbackRequestWithDeleteDataAndLogFilesAction(partitionPath, instantToRollback);
|
||||
@@ -350,34 +408,6 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
|
||||
}
|
||||
}
|
||||
|
||||
private List<HoodieCleanStat> cleanPartitionPaths(List<String> partitionsToClean, JavaSparkContext jsc) {
|
||||
int cleanerParallelism = Math.min(partitionsToClean.size(), config.getCleanerParallelism());
|
||||
logger.info("Using cleanerParallelism: " + cleanerParallelism);
|
||||
List<Tuple2<String, PartitionCleanStat>> partitionCleanStats = jsc
|
||||
.parallelize(partitionsToClean, cleanerParallelism).flatMapToPair(getFilesToDeleteFunc(this, config))
|
||||
.repartition(cleanerParallelism) // repartition to remove skews
|
||||
.mapPartitionsToPair(deleteFilesFunc(this)).reduceByKey(
|
||||
// merge partition level clean stats below
|
||||
(Function2<PartitionCleanStat, PartitionCleanStat, PartitionCleanStat>) (e1, e2) -> e1.merge(e2))
|
||||
.collect();
|
||||
|
||||
Map<String, PartitionCleanStat> partitionCleanStatsMap =
|
||||
partitionCleanStats.stream().collect(Collectors.toMap(Tuple2::_1, Tuple2::_2));
|
||||
|
||||
HoodieCleanHelper cleaner = new HoodieCleanHelper(this, config);
|
||||
// Return PartitionCleanStat for each partition passed.
|
||||
return partitionsToClean.stream().map(partitionPath -> {
|
||||
PartitionCleanStat partitionCleanStat =
|
||||
(partitionCleanStatsMap.containsKey(partitionPath)) ? partitionCleanStatsMap.get(partitionPath)
|
||||
: new PartitionCleanStat(partitionPath);
|
||||
return HoodieCleanStat.newBuilder().withPolicy(config.getCleanerPolicy()).withPartitionPath(partitionPath)
|
||||
.withEarliestCommitRetained(cleaner.getEarliestCommitToRetain())
|
||||
.withDeletePathPattern(partitionCleanStat.deletePathPatterns)
|
||||
.withSuccessfulDeletes(partitionCleanStat.successDeleteFiles)
|
||||
.withFailedDeletes(partitionCleanStat.failedDeleteFiles).build();
|
||||
}).collect(Collectors.toList());
|
||||
}
|
||||
|
||||
enum BucketType {
|
||||
UPDATE, INSERT
|
||||
}
|
||||
|
||||
@@ -185,12 +185,12 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends Hoodi
|
||||
logger.info("Unpublished " + commit);
|
||||
Long startTime = System.currentTimeMillis();
|
||||
List<RollbackRequest> rollbackRequests = generateRollbackRequests(jsc, instantToRollback);
|
||||
//TODO: We need to persist this as rollback workload and use it in case of partial failures
|
||||
// TODO: We need to persist this as rollback workload and use it in case of partial failures
|
||||
List<HoodieRollbackStat> allRollbackStats =
|
||||
new RollbackExecutor(metaClient, config).performRollback(jsc, instantToRollback, rollbackRequests);
|
||||
// Delete Inflight instants if enabled
|
||||
deleteInflightInstant(deleteInstants, this.getActiveTimeline(), new HoodieInstant(true, instantToRollback
|
||||
.getAction(), instantToRollback.getTimestamp()));
|
||||
deleteInflightInstant(deleteInstants, this.getActiveTimeline(),
|
||||
new HoodieInstant(true, instantToRollback.getAction(), instantToRollback.getTimestamp()));
|
||||
|
||||
logger.info("Time(in ms) taken to finish rollback " + (System.currentTimeMillis() - startTime));
|
||||
|
||||
@@ -200,6 +200,7 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends Hoodi
|
||||
/**
|
||||
* Generate all rollback requests that we need to perform for rolling back this action without actually performing
|
||||
* rolling back
|
||||
*
|
||||
* @param jsc JavaSparkContext
|
||||
* @param instantToRollback Instant to Rollback
|
||||
* @return list of rollback requests
|
||||
@@ -211,31 +212,30 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends Hoodi
|
||||
List<String> partitions = FSUtils.getAllPartitionPaths(this.metaClient.getFs(), this.getMetaClient().getBasePath(),
|
||||
config.shouldAssumeDatePartitioning());
|
||||
int sparkPartitions = Math.max(Math.min(partitions.size(), config.getRollbackParallelism()), 1);
|
||||
return jsc.parallelize(partitions, Math.min(partitions.size(), sparkPartitions))
|
||||
.flatMap(partitionPath -> {
|
||||
return jsc.parallelize(partitions, Math.min(partitions.size(), sparkPartitions)).flatMap(partitionPath -> {
|
||||
HoodieActiveTimeline activeTimeline = this.getActiveTimeline().reload();
|
||||
List<RollbackRequest> partitionRollbackRequests = new ArrayList<>();
|
||||
switch (instantToRollback.getAction()) {
|
||||
case HoodieTimeline.COMMIT_ACTION:
|
||||
logger.info("Rolling back commit action. There are higher delta commits. So only rolling back this "
|
||||
+ "instant");
|
||||
partitionRollbackRequests.add(RollbackRequest.createRollbackRequestWithDeleteDataAndLogFilesAction(
|
||||
partitionPath, instantToRollback));
|
||||
logger.info(
|
||||
"Rolling back commit action. There are higher delta commits. So only rolling back this " + "instant");
|
||||
partitionRollbackRequests.add(
|
||||
RollbackRequest.createRollbackRequestWithDeleteDataAndLogFilesAction(partitionPath, instantToRollback));
|
||||
break;
|
||||
case HoodieTimeline.COMPACTION_ACTION:
|
||||
// If there is no delta commit present after the current commit (if compaction), no action, else we
|
||||
// need to make sure that a compaction commit rollback also deletes any log files written as part of the
|
||||
// succeeding deltacommit.
|
||||
boolean higherDeltaCommits = !activeTimeline.getDeltaCommitTimeline()
|
||||
.filterCompletedInstants().findInstantsAfter(commit, 1).empty();
|
||||
boolean higherDeltaCommits =
|
||||
!activeTimeline.getDeltaCommitTimeline().filterCompletedInstants().findInstantsAfter(commit, 1).empty();
|
||||
if (higherDeltaCommits) {
|
||||
// Rollback of a compaction action with no higher deltacommit means that the compaction is scheduled
|
||||
// and has not yet finished. In this scenario we should delete only the newly created parquet files
|
||||
// and not corresponding base commit log files created with this as baseCommit since updates would
|
||||
// have been written to the log files.
|
||||
logger.info("Rolling back compaction. There are higher delta commits. So only deleting data files");
|
||||
partitionRollbackRequests.add(RollbackRequest.createRollbackRequestWithDeleteDataFilesOnlyAction(
|
||||
partitionPath, instantToRollback));
|
||||
partitionRollbackRequests.add(
|
||||
RollbackRequest.createRollbackRequestWithDeleteDataFilesOnlyAction(partitionPath, instantToRollback));
|
||||
} else {
|
||||
// No deltacommits present after this compaction commit (inflight or requested). In this case, we
|
||||
// can also delete any log files that were created with this compaction commit as base
|
||||
@@ -243,8 +243,7 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends Hoodi
|
||||
logger.info("Rolling back compaction plan. There are NO higher delta commits. So deleting both data and"
|
||||
+ " log files");
|
||||
partitionRollbackRequests.add(
|
||||
RollbackRequest.createRollbackRequestWithDeleteDataAndLogFilesAction(partitionPath,
|
||||
instantToRollback));
|
||||
RollbackRequest.createRollbackRequestWithDeleteDataAndLogFilesAction(partitionPath, instantToRollback));
|
||||
}
|
||||
break;
|
||||
case HoodieTimeline.DELTA_COMMIT_ACTION:
|
||||
@@ -273,15 +272,17 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends Hoodi
|
||||
// as well if the base parquet file gets deleted.
|
||||
try {
|
||||
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes(
|
||||
metaClient.getCommitTimeline().getInstantDetails(
|
||||
metaClient.getCommitTimeline()
|
||||
.getInstantDetails(
|
||||
new HoodieInstant(true, instantToRollback.getAction(), instantToRollback.getTimestamp()))
|
||||
.get(), HoodieCommitMetadata.class);
|
||||
.get(),
|
||||
HoodieCommitMetadata.class);
|
||||
|
||||
// In case all data was inserts and the commit failed, delete the file belonging to that commit
|
||||
// We do not know fileIds for inserts (first inserts are either log files or parquet files),
|
||||
// delete all files for the corresponding failed commit, if present (same as COW)
|
||||
partitionRollbackRequests.add(RollbackRequest.createRollbackRequestWithDeleteDataAndLogFilesAction(
|
||||
partitionPath, instantToRollback));
|
||||
partitionRollbackRequests.add(
|
||||
RollbackRequest.createRollbackRequestWithDeleteDataAndLogFilesAction(partitionPath, instantToRollback));
|
||||
|
||||
// append rollback blocks for updates
|
||||
if (commitMetadata.getPartitionToWriteStats().containsKey(partitionPath)) {
|
||||
@@ -428,8 +429,7 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends Hoodi
|
||||
// baseCommit always by listing the file slice
|
||||
Map<String, String> fileIdToBaseCommitTimeForLogMap = this.getRTFileSystemView().getLatestFileSlices(partitionPath)
|
||||
.collect(Collectors.toMap(FileSlice::getFileId, FileSlice::getBaseInstantTime));
|
||||
return commitMetadata.getPartitionToWriteStats().get(partitionPath).stream()
|
||||
.filter(wStat -> {
|
||||
return commitMetadata.getPartitionToWriteStats().get(partitionPath).stream().filter(wStat -> {
|
||||
|
||||
// Filter out stats without prevCommit since they are all inserts
|
||||
boolean validForRollback = (wStat != null) && (wStat.getPrevCommit() != HoodieWriteStat.NULL_COMMIT)
|
||||
@@ -437,8 +437,9 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends Hoodi
|
||||
|
||||
if (validForRollback) {
|
||||
// For sanity, log instant time can never be less than base-commit on which we are rolling back
|
||||
Preconditions.checkArgument(HoodieTimeline.compareTimestamps(fileIdToBaseCommitTimeForLogMap.get(
|
||||
wStat.getFileId()), rollbackInstant.getTimestamp(), HoodieTimeline.LESSER_OR_EQUAL));
|
||||
Preconditions
|
||||
.checkArgument(HoodieTimeline.compareTimestamps(fileIdToBaseCommitTimeForLogMap.get(wStat.getFileId()),
|
||||
rollbackInstant.getTimestamp(), HoodieTimeline.LESSER_OR_EQUAL));
|
||||
}
|
||||
|
||||
return validForRollback && HoodieTimeline.compareTimestamps(fileIdToBaseCommitTimeForLogMap.get(
|
||||
|
||||
@@ -31,6 +31,7 @@ import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.WriteStatus;
|
||||
import org.apache.hudi.avro.model.HoodieCleanerPlan;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionPlan;
|
||||
import org.apache.hudi.avro.model.HoodieSavepointMetadata;
|
||||
import org.apache.hudi.client.utils.ClientUtils;
|
||||
@@ -190,6 +191,13 @@ public abstract class HoodieTable<T extends HoodieRecordPayload> implements Seri
|
||||
return getActiveTimeline().getCleanerTimeline().filterCompletedInstants();
|
||||
}
|
||||
|
||||
/**
|
||||
* Get clean timeline
|
||||
*/
|
||||
public HoodieTimeline getCleanTimeline() {
|
||||
return getActiveTimeline().getCleanerTimeline();
|
||||
}
|
||||
|
||||
/**
|
||||
* Get only the completed (no-inflights) savepoint timeline
|
||||
*/
|
||||
@@ -265,9 +273,21 @@ public abstract class HoodieTable<T extends HoodieRecordPayload> implements Seri
|
||||
HoodieCompactionPlan compactionPlan);
|
||||
|
||||
/**
|
||||
* Clean partition paths according to cleaning policy and returns the number of files cleaned.
|
||||
* Generates list of files that are eligible for cleaning
|
||||
*
|
||||
* @param jsc Java Spark Context
|
||||
* @return Cleaner Plan containing list of files to be deleted.
|
||||
*/
|
||||
public abstract List<HoodieCleanStat> clean(JavaSparkContext jsc);
|
||||
public abstract HoodieCleanerPlan scheduleClean(JavaSparkContext jsc);
|
||||
|
||||
/**
|
||||
* Cleans the files listed in the cleaner plan associated with clean instant
|
||||
*
|
||||
* @param jsc Java Spark Context
|
||||
* @param cleanInstant Clean Instant
|
||||
* @return list of Clean Stats
|
||||
*/
|
||||
public abstract List<HoodieCleanStat> clean(JavaSparkContext jsc, HoodieInstant cleanInstant);
|
||||
|
||||
/**
|
||||
* Rollback the (inflight/committed) record changes with the given commit time. Four steps: (1) Atomically unpublish
|
||||
|
||||
@@ -65,8 +65,8 @@ public class RollbackExecutor implements Serializable {
|
||||
/**
|
||||
* Performs all rollback actions that we have collected in parallel.
|
||||
*/
|
||||
public List<HoodieRollbackStat> performRollback(JavaSparkContext jsc,
|
||||
HoodieInstant instantToRollback, List<RollbackRequest> rollbackRequests) {
|
||||
public List<HoodieRollbackStat> performRollback(JavaSparkContext jsc, HoodieInstant instantToRollback,
|
||||
List<RollbackRequest> rollbackRequests) {
|
||||
|
||||
SerializablePathFilter filter = (path) -> {
|
||||
if (path.toString().contains(".parquet")) {
|
||||
@@ -101,11 +101,10 @@ public class RollbackExecutor implements Serializable {
|
||||
Writer writer = null;
|
||||
boolean success = false;
|
||||
try {
|
||||
writer = HoodieLogFormat.newWriterBuilder().onParentPath(
|
||||
FSUtils.getPartitionPath(metaClient.getBasePath(), rollbackRequest.getPartitionPath()))
|
||||
writer = HoodieLogFormat.newWriterBuilder()
|
||||
.onParentPath(FSUtils.getPartitionPath(metaClient.getBasePath(), rollbackRequest.getPartitionPath()))
|
||||
.withFileId(rollbackRequest.getFileId().get())
|
||||
.overBaseCommit(rollbackRequest.getLatestBaseInstant().get())
|
||||
.withFs(metaClient.getFs())
|
||||
.overBaseCommit(rollbackRequest.getLatestBaseInstant().get()).withFs(metaClient.getFs())
|
||||
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).build();
|
||||
|
||||
// generate metadata
|
||||
@@ -114,8 +113,7 @@ public class RollbackExecutor implements Serializable {
|
||||
writer = writer.appendBlock(new HoodieCommandBlock(header));
|
||||
success = true;
|
||||
} catch (IOException | InterruptedException io) {
|
||||
throw new HoodieRollbackException(
|
||||
"Failed to rollback for instant " + instantToRollback, io);
|
||||
throw new HoodieRollbackException("Failed to rollback for instant " + instantToRollback, io);
|
||||
} finally {
|
||||
try {
|
||||
if (writer != null) {
|
||||
@@ -130,8 +128,7 @@ public class RollbackExecutor implements Serializable {
|
||||
// getFileStatus would reflect correct stats and FileNotFoundException is not thrown in
|
||||
// cloud-storage : HUDI-168
|
||||
Map<FileStatus, Long> filesToNumBlocksRollback = new HashMap<>();
|
||||
filesToNumBlocksRollback.put(metaClient.getFs()
|
||||
.getFileStatus(writer.getLogFile().getPath()), 1L);
|
||||
filesToNumBlocksRollback.put(metaClient.getFs().getFileStatus(writer.getLogFile().getPath()), 1L);
|
||||
return new Tuple2<String, HoodieRollbackStat>(rollbackRequest.getPartitionPath(),
|
||||
HoodieRollbackStat.newBuilder().withPartitionPath(rollbackRequest.getPartitionPath())
|
||||
.withRollbackBlockAppendResults(filesToNumBlocksRollback).build());
|
||||
@@ -180,8 +177,7 @@ public class RollbackExecutor implements Serializable {
|
||||
* Common method used for cleaning out parquet files under a partition path during rollback of a set of commits
|
||||
*/
|
||||
private Map<FileStatus, Boolean> deleteCleanedFiles(HoodieTableMetaClient metaClient, HoodieWriteConfig config,
|
||||
Map<FileStatus, Boolean> results, String partitionPath,
|
||||
PathFilter filter) throws IOException {
|
||||
Map<FileStatus, Boolean> results, String partitionPath, PathFilter filter) throws IOException {
|
||||
logger.info("Cleaning path " + partitionPath);
|
||||
FileSystem fs = metaClient.getFs();
|
||||
FileStatus[] toBeDeleted = fs.listStatus(FSUtils.getPartitionPath(config.getBasePath(), partitionPath), filter);
|
||||
|
||||
@@ -30,9 +30,7 @@ public class RollbackRequest {
|
||||
* Rollback Action Types
|
||||
*/
|
||||
public enum RollbackAction {
|
||||
DELETE_DATA_FILES_ONLY,
|
||||
DELETE_DATA_AND_LOG_FILES,
|
||||
APPEND_ROLLBACK_BLOCK
|
||||
DELETE_DATA_FILES_ONLY, DELETE_DATA_AND_LOG_FILES, APPEND_ROLLBACK_BLOCK
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -60,8 +58,8 @@ public class RollbackRequest {
|
||||
*/
|
||||
private final RollbackAction rollbackAction;
|
||||
|
||||
public RollbackRequest(String partitionPath, HoodieInstant rollbackInstant,
|
||||
Option<String> fileId, Option<String> latestBaseInstant, RollbackAction rollbackAction) {
|
||||
public RollbackRequest(String partitionPath, HoodieInstant rollbackInstant, Option<String> fileId,
|
||||
Option<String> latestBaseInstant, RollbackAction rollbackAction) {
|
||||
this.partitionPath = partitionPath;
|
||||
this.rollbackInstant = rollbackInstant;
|
||||
this.fileId = fileId;
|
||||
|
||||
@@ -21,6 +21,7 @@ import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.Executors;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.LocalFileSystem;
|
||||
@@ -52,6 +53,11 @@ public abstract class HoodieClientTestHarness extends HoodieCommonTestHarness im
|
||||
protected transient HoodieTestDataGenerator dataGen = null;
|
||||
protected transient ExecutorService executorService;
|
||||
protected transient HoodieTableMetaClient metaClient;
|
||||
private static AtomicInteger instantGen = new AtomicInteger(1);
|
||||
|
||||
public String getNextInstant() {
|
||||
return String.format("%09d", instantGen.getAndIncrement());
|
||||
}
|
||||
|
||||
// dfs
|
||||
protected String dfsBasePath;
|
||||
|
||||
@@ -30,6 +30,7 @@ import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
import java.util.Set;
|
||||
import java.util.TreeSet;
|
||||
import java.util.function.Predicate;
|
||||
@@ -37,6 +38,8 @@ import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.RemoteIterator;
|
||||
import org.apache.hudi.avro.model.HoodieCleanMetadata;
|
||||
import org.apache.hudi.avro.model.HoodieCleanPartitionMetadata;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionPlan;
|
||||
import org.apache.hudi.common.HoodieCleanStat;
|
||||
import org.apache.hudi.common.HoodieTestDataGenerator;
|
||||
@@ -69,12 +72,8 @@ import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.scheduler.SparkListener;
|
||||
import org.apache.spark.scheduler.SparkListenerTaskEnd;
|
||||
import org.apache.spark.util.AccumulatorV2;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
import scala.collection.Iterator;
|
||||
|
||||
/**
|
||||
* Test Cleaning related logic
|
||||
@@ -396,6 +395,62 @@ public class TestCleaner extends TestHoodieClientBase {
|
||||
});
|
||||
}
|
||||
|
||||
/**
|
||||
* Helper to run cleaner and collect Clean Stats
|
||||
*
|
||||
* @param config HoodieWriteConfig
|
||||
*/
|
||||
private List<HoodieCleanStat> runCleaner(HoodieWriteConfig config) {
|
||||
return runCleaner(config, false);
|
||||
}
|
||||
|
||||
/**
|
||||
* Helper to run cleaner and collect Clean Stats
|
||||
*
|
||||
* @param config HoodieWriteConfig
|
||||
*/
|
||||
private List<HoodieCleanStat> runCleaner(HoodieWriteConfig config, boolean simulateRetryFailure) {
|
||||
HoodieCleanClient writeClient = getHoodieCleanClient(config);
|
||||
|
||||
String cleanInstantTs = getNextInstant();
|
||||
HoodieCleanMetadata cleanMetadata1 = writeClient.clean(cleanInstantTs);
|
||||
|
||||
if (null == cleanMetadata1) {
|
||||
return new ArrayList<>();
|
||||
}
|
||||
|
||||
if (simulateRetryFailure) {
|
||||
metaClient.reloadActiveTimeline()
|
||||
.revertToInflight(new HoodieInstant(State.COMPLETED, HoodieTimeline.CLEAN_ACTION, cleanInstantTs));
|
||||
final HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc);
|
||||
HoodieCleanMetadata cleanMetadata2 = writeClient.runClean(table, cleanInstantTs);
|
||||
Assert.assertTrue(
|
||||
Objects.equals(cleanMetadata1.getEarliestCommitToRetain(), cleanMetadata2.getEarliestCommitToRetain()));
|
||||
Assert.assertEquals(new Integer(0), cleanMetadata2.getTotalFilesDeleted());
|
||||
Assert.assertEquals(cleanMetadata1.getPartitionMetadata().keySet(),
|
||||
cleanMetadata2.getPartitionMetadata().keySet());
|
||||
cleanMetadata1.getPartitionMetadata().keySet().stream().forEach(k -> {
|
||||
HoodieCleanPartitionMetadata p1 = cleanMetadata1.getPartitionMetadata().get(k);
|
||||
HoodieCleanPartitionMetadata p2 = cleanMetadata2.getPartitionMetadata().get(k);
|
||||
Assert.assertEquals(p1.getDeletePathPatterns(), p2.getDeletePathPatterns());
|
||||
Assert.assertEquals(p1.getSuccessDeleteFiles(), p2.getFailedDeleteFiles());
|
||||
Assert.assertEquals(p1.getPartitionPath(), p2.getPartitionPath());
|
||||
Assert.assertEquals(k, p1.getPartitionPath());
|
||||
});
|
||||
}
|
||||
List<HoodieCleanStat> stats = cleanMetadata1.getPartitionMetadata().values().stream()
|
||||
.map(x -> new HoodieCleanStat.Builder().withPartitionPath(x.getPartitionPath())
|
||||
.withFailedDeletes(x.getFailedDeleteFiles()).withSuccessfulDeletes(x.getSuccessDeleteFiles())
|
||||
.withPolicy(HoodieCleaningPolicy.valueOf(x.getPolicy())).withDeletePathPattern(x.getDeletePathPatterns())
|
||||
.withEarliestCommitRetained(Option.ofNullable(cleanMetadata1.getEarliestCommitToRetain() != null
|
||||
? new HoodieInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "000")
|
||||
: null))
|
||||
.build())
|
||||
.collect(Collectors.toList());
|
||||
|
||||
return stats;
|
||||
}
|
||||
|
||||
/**
|
||||
* Test HoodieTable.clean() Cleaning by versions logic
|
||||
*/
|
||||
@@ -417,7 +472,7 @@ public class TestCleaner extends TestHoodieClientBase {
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc);
|
||||
|
||||
List<HoodieCleanStat> hoodieCleanStatsOne = table.clean(jsc);
|
||||
List<HoodieCleanStat> hoodieCleanStatsOne = runCleaner(config);
|
||||
assertEquals("Must not clean any files", 0,
|
||||
getCleanStat(hoodieCleanStatsOne, HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH).getSuccessDeleteFiles()
|
||||
.size());
|
||||
@@ -441,7 +496,7 @@ public class TestCleaner extends TestHoodieClientBase {
|
||||
HoodieTestUtils.createDataFile(basePath, HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, "001", file1P0C0); // update
|
||||
HoodieTestUtils.createDataFile(basePath, HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH, "001", file1P1C0); // update
|
||||
|
||||
List<HoodieCleanStat> hoodieCleanStatsTwo = table.clean(jsc);
|
||||
List<HoodieCleanStat> hoodieCleanStatsTwo = runCleaner(config);;
|
||||
assertEquals("Must clean 1 file", 1,
|
||||
getCleanStat(hoodieCleanStatsTwo, HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH).getSuccessDeleteFiles()
|
||||
.size());
|
||||
@@ -467,7 +522,7 @@ public class TestCleaner extends TestHoodieClientBase {
|
||||
String file3P0C2 =
|
||||
HoodieTestUtils.createNewDataFile(basePath, HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, "002");
|
||||
|
||||
List<HoodieCleanStat> hoodieCleanStatsThree = table.clean(jsc);
|
||||
List<HoodieCleanStat> hoodieCleanStatsThree = runCleaner(config);;
|
||||
assertEquals("Must clean two files", 2,
|
||||
getCleanStat(hoodieCleanStatsThree, HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH)
|
||||
.getSuccessDeleteFiles().size());
|
||||
@@ -480,7 +535,7 @@ public class TestCleaner extends TestHoodieClientBase {
|
||||
|
||||
// No cleaning on partially written file, with no commit.
|
||||
HoodieTestUtils.createDataFile(basePath, HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, "003", file3P0C2); // update
|
||||
List<HoodieCleanStat> hoodieCleanStatsFour = table.clean(jsc);
|
||||
List<HoodieCleanStat> hoodieCleanStatsFour = runCleaner(config);
|
||||
assertEquals("Must not clean any files", 0,
|
||||
getCleanStat(hoodieCleanStatsFour, HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH).getSuccessDeleteFiles()
|
||||
.size());
|
||||
@@ -525,7 +580,7 @@ public class TestCleaner extends TestHoodieClientBase {
|
||||
HoodieTestUtils.createCompactionCommitFiles(fs, basePath, "001");
|
||||
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc);
|
||||
List<HoodieCleanStat> hoodieCleanStats = table.clean(jsc);
|
||||
List<HoodieCleanStat> hoodieCleanStats = runCleaner(config);;
|
||||
assertEquals("Must clean three files, one parquet and 2 log files", 3,
|
||||
getCleanStat(hoodieCleanStats, HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH).getSuccessDeleteFiles()
|
||||
.size());
|
||||
@@ -542,6 +597,22 @@ public class TestCleaner extends TestHoodieClientBase {
|
||||
*/
|
||||
@Test
|
||||
public void testKeepLatestCommits() throws IOException {
|
||||
testKeepLatestCommits(false);
|
||||
}
|
||||
|
||||
/**
|
||||
* Test HoodieTable.clean() Cleaning by commit logic for MOR table with Log files. Here the operations are simulated
|
||||
* such that first clean attempt failed after files were cleaned and a subsequent cleanup succeeds.
|
||||
*/
|
||||
@Test
|
||||
public void testKeepLatestCommitsWithFailureRetry() throws IOException {
|
||||
testKeepLatestCommits(true);
|
||||
}
|
||||
|
||||
/**
|
||||
* Test HoodieTable.clean() Cleaning by commit logic for MOR table with Log files.
|
||||
*/
|
||||
private void testKeepLatestCommits(boolean simulateFailureRetry) throws IOException {
|
||||
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).withAssumeDatePartitioning(true)
|
||||
.withCompactionConfig(HoodieCompactionConfig.newBuilder()
|
||||
.withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(2).build())
|
||||
@@ -558,7 +629,7 @@ public class TestCleaner extends TestHoodieClientBase {
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc);
|
||||
|
||||
List<HoodieCleanStat> hoodieCleanStatsOne = table.clean(jsc);
|
||||
List<HoodieCleanStat> hoodieCleanStatsOne = runCleaner(config, simulateFailureRetry);
|
||||
assertEquals("Must not clean any files", 0,
|
||||
getCleanStat(hoodieCleanStatsOne, HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH).getSuccessDeleteFiles()
|
||||
.size());
|
||||
@@ -582,7 +653,7 @@ public class TestCleaner extends TestHoodieClientBase {
|
||||
HoodieTestUtils.createDataFile(basePath, HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, "001", file1P0C0); // update
|
||||
HoodieTestUtils.createDataFile(basePath, HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH, "001", file1P1C0); // update
|
||||
|
||||
List<HoodieCleanStat> hoodieCleanStatsTwo = table.clean(jsc);
|
||||
List<HoodieCleanStat> hoodieCleanStatsTwo = runCleaner(config, simulateFailureRetry);
|
||||
assertEquals("Must not clean any files", 0,
|
||||
getCleanStat(hoodieCleanStatsTwo, HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH).getSuccessDeleteFiles()
|
||||
.size());
|
||||
@@ -608,7 +679,7 @@ public class TestCleaner extends TestHoodieClientBase {
|
||||
String file3P0C2 =
|
||||
HoodieTestUtils.createNewDataFile(basePath, HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, "002");
|
||||
|
||||
List<HoodieCleanStat> hoodieCleanStatsThree = table.clean(jsc);
|
||||
List<HoodieCleanStat> hoodieCleanStatsThree = runCleaner(config, simulateFailureRetry);
|
||||
assertEquals("Must not clean any file. We have to keep 1 version before the latest commit time to keep", 0,
|
||||
getCleanStat(hoodieCleanStatsThree, HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH)
|
||||
.getSuccessDeleteFiles().size());
|
||||
@@ -626,7 +697,7 @@ public class TestCleaner extends TestHoodieClientBase {
|
||||
String file4P0C3 =
|
||||
HoodieTestUtils.createNewDataFile(basePath, HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, "003");
|
||||
|
||||
List<HoodieCleanStat> hoodieCleanStatsFour = table.clean(jsc);
|
||||
List<HoodieCleanStat> hoodieCleanStatsFour = runCleaner(config, simulateFailureRetry);
|
||||
assertEquals("Must not clean one old file", 1,
|
||||
getCleanStat(hoodieCleanStatsFour, HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH).getSuccessDeleteFiles()
|
||||
.size());
|
||||
@@ -648,7 +719,7 @@ public class TestCleaner extends TestHoodieClientBase {
|
||||
|
||||
// No cleaning on partially written file, with no commit.
|
||||
HoodieTestUtils.createDataFile(basePath, HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, "004", file3P0C2); // update
|
||||
List<HoodieCleanStat> hoodieCleanStatsFive = table.clean(jsc);
|
||||
List<HoodieCleanStat> hoodieCleanStatsFive = runCleaner(config, simulateFailureRetry);
|
||||
assertEquals("Must not clean any files", 0,
|
||||
getCleanStat(hoodieCleanStatsFive, HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH).getSuccessDeleteFiles()
|
||||
.size());
|
||||
@@ -694,88 +765,10 @@ public class TestCleaner extends TestHoodieClientBase {
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc);
|
||||
|
||||
List<HoodieCleanStat> hoodieCleanStatsOne = table.clean(jsc);
|
||||
List<HoodieCleanStat> hoodieCleanStatsOne = runCleaner(config);
|
||||
assertTrue("HoodieCleanStats should be empty for a table with empty partitionPaths", hoodieCleanStatsOne.isEmpty());
|
||||
}
|
||||
|
||||
/**
|
||||
* Test Clean-by-commits behavior in the presence of skewed partitions
|
||||
*/
|
||||
@Test
|
||||
public void testCleaningSkewedPartitons() throws IOException {
|
||||
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).withAssumeDatePartitioning(true)
|
||||
.withCompactionConfig(HoodieCompactionConfig.newBuilder()
|
||||
.withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(2).build())
|
||||
.build();
|
||||
Map<Long, Long> stageOneShuffleReadTaskRecordsCountMap = new HashMap<>();
|
||||
|
||||
// Since clean involves repartition in order to uniformly distribute data,
|
||||
// we can inspect the number of records read by various tasks in stage 1.
|
||||
// There should not be skew in the number of records read in the task.
|
||||
|
||||
// SparkListener below listens to the stage end events and captures number of
|
||||
// records read by various tasks in stage-1.
|
||||
jsc.sc().addSparkListener(new SparkListener() {
|
||||
|
||||
@Override
|
||||
public void onTaskEnd(SparkListenerTaskEnd taskEnd) {
|
||||
|
||||
Iterator<AccumulatorV2<?, ?>> iterator = taskEnd.taskMetrics().accumulators().iterator();
|
||||
while (iterator.hasNext()) {
|
||||
AccumulatorV2 accumulator = iterator.next();
|
||||
if (taskEnd.stageId() == 1 && accumulator.isRegistered() && accumulator.name().isDefined()
|
||||
&& accumulator.name().get().equals("internal.metrics.shuffle.read.recordsRead")) {
|
||||
stageOneShuffleReadTaskRecordsCountMap.put(taskEnd.taskInfo().taskId(), (Long) accumulator.value());
|
||||
}
|
||||
}
|
||||
}
|
||||
});
|
||||
|
||||
// make 1 commit, with 100 files in one partition and 10 in other two
|
||||
HoodieTestUtils.createCommitFiles(basePath, "000");
|
||||
List<String> filesP0C0 = createFilesInPartition(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, "000", 100);
|
||||
List<String> filesP1C0 = createFilesInPartition(HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH, "000", 10);
|
||||
List<String> filesP2C0 = createFilesInPartition(HoodieTestDataGenerator.DEFAULT_THIRD_PARTITION_PATH, "000", 10);
|
||||
|
||||
HoodieTestUtils.createCommitFiles(basePath, "001");
|
||||
updateAllFilesInPartition(filesP0C0, HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, "001");
|
||||
updateAllFilesInPartition(filesP1C0, HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH, "001");
|
||||
updateAllFilesInPartition(filesP2C0, HoodieTestDataGenerator.DEFAULT_THIRD_PARTITION_PATH, "001");
|
||||
|
||||
HoodieTestUtils.createCommitFiles(basePath, "002");
|
||||
updateAllFilesInPartition(filesP0C0, HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, "002");
|
||||
updateAllFilesInPartition(filesP1C0, HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH, "002");
|
||||
updateAllFilesInPartition(filesP2C0, HoodieTestDataGenerator.DEFAULT_THIRD_PARTITION_PATH, "002");
|
||||
|
||||
HoodieTestUtils.createCommitFiles(basePath, "003");
|
||||
updateAllFilesInPartition(filesP0C0, HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, "003");
|
||||
updateAllFilesInPartition(filesP1C0, HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH, "003");
|
||||
updateAllFilesInPartition(filesP2C0, HoodieTestDataGenerator.DEFAULT_THIRD_PARTITION_PATH, "003");
|
||||
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc);
|
||||
List<HoodieCleanStat> hoodieCleanStats = table.clean(jsc);
|
||||
|
||||
assertEquals(100, getCleanStat(hoodieCleanStats, HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH)
|
||||
.getSuccessDeleteFiles().size());
|
||||
assertEquals(10, getCleanStat(hoodieCleanStats, HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH)
|
||||
.getSuccessDeleteFiles().size());
|
||||
assertEquals(10, getCleanStat(hoodieCleanStats, HoodieTestDataGenerator.DEFAULT_THIRD_PARTITION_PATH)
|
||||
.getSuccessDeleteFiles().size());
|
||||
|
||||
// 3 tasks are expected since the number of partitions is 3
|
||||
assertEquals(3, stageOneShuffleReadTaskRecordsCountMap.keySet().size());
|
||||
// Sum of all records processed = total number of files to clean
|
||||
assertEquals(120,
|
||||
stageOneShuffleReadTaskRecordsCountMap.values().stream().reduce((a, b) -> a + b).get().intValue());
|
||||
assertTrue(
|
||||
"The skew in handling files to clean is not removed. "
|
||||
+ "Each task should handle more records than the partitionPath with least files "
|
||||
+ "and less records than the partitionPath with most files.",
|
||||
stageOneShuffleReadTaskRecordsCountMap.values().stream().filter(a -> a > 10 && a < 100).count() == 3);
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Test Keep Latest Commits when there are pending compactions
|
||||
*/
|
||||
@@ -794,14 +787,28 @@ public class TestCleaner extends TestHoodieClientBase {
|
||||
// FileId3 1 2 3 001
|
||||
// FileId2 0 0 0 000
|
||||
// FileId1 0 0 0 000
|
||||
testPendingCompactions(config, 48, 18);
|
||||
testPendingCompactions(config, 48, 18, false);
|
||||
}
|
||||
|
||||
/**
|
||||
* Test HoodieTable.clean() Cleaning by commit logic for MOR table with Log files. Here the operations are simulated
|
||||
* such that first clean attempt failed after files were cleaned and a subsequent cleanup succeeds.
|
||||
*/
|
||||
@Test
|
||||
public void testKeepLatestVersionsWithPendingCompactions() throws IOException {
|
||||
testKeepLatestVersionsWithPendingCompactions(false);
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Test Keep Latest Versions when there are pending compactions
|
||||
*/
|
||||
@Test
|
||||
public void testKeepLatestVersionsWithPendingCompactions() throws IOException {
|
||||
public void testKeepLatestVersionsWithPendingCompactionsAndFailureRetry() throws IOException {
|
||||
testKeepLatestVersionsWithPendingCompactions(true);
|
||||
}
|
||||
|
||||
private void testKeepLatestVersionsWithPendingCompactions(boolean retryFailure) throws IOException {
|
||||
HoodieWriteConfig config =
|
||||
HoodieWriteConfig.newBuilder().withPath(basePath).withAssumeDatePartitioning(true)
|
||||
.withCompactionConfig(HoodieCompactionConfig.newBuilder()
|
||||
@@ -816,7 +823,7 @@ public class TestCleaner extends TestHoodieClientBase {
|
||||
// FileId3 0 0 0 000, 001
|
||||
// FileId2 0 0 0 000
|
||||
// FileId1 0 0 0 000
|
||||
testPendingCompactions(config, 36, 9);
|
||||
testPendingCompactions(config, 36, 9, retryFailure);
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -825,8 +832,8 @@ public class TestCleaner extends TestHoodieClientBase {
|
||||
* @param config Hoodie Write Config
|
||||
* @param expNumFilesDeleted Number of files deleted
|
||||
*/
|
||||
public void testPendingCompactions(HoodieWriteConfig config, int expNumFilesDeleted,
|
||||
int expNumFilesUnderCompactionDeleted) throws IOException {
|
||||
private void testPendingCompactions(HoodieWriteConfig config, int expNumFilesDeleted,
|
||||
int expNumFilesUnderCompactionDeleted, boolean retryFailure) throws IOException {
|
||||
HoodieTableMetaClient metaClient =
|
||||
HoodieTestUtils.init(jsc.hadoopConfiguration(), basePath, HoodieTableType.MERGE_ON_READ);
|
||||
String[] instants = new String[] {"000", "001", "003", "005", "007", "009", "011", "013"};
|
||||
@@ -897,7 +904,7 @@ public class TestCleaner extends TestHoodieClientBase {
|
||||
// Clean now
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc);
|
||||
List<HoodieCleanStat> hoodieCleanStats = table.clean(jsc);
|
||||
List<HoodieCleanStat> hoodieCleanStats = runCleaner(config, retryFailure);
|
||||
|
||||
// Test for safety
|
||||
final HoodieTableMetaClient newMetaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
|
||||
@@ -52,6 +52,7 @@ import org.apache.hudi.config.HoodieStorageConfig;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.index.HoodieIndex;
|
||||
import org.apache.hudi.index.HoodieIndex.IndexType;
|
||||
import org.apache.hudi.metrics.HoodieMetrics;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
@@ -77,6 +78,10 @@ public class TestHoodieClientBase extends HoodieClientTestHarness {
|
||||
cleanupResources();
|
||||
}
|
||||
|
||||
protected HoodieCleanClient getHoodieCleanClient(HoodieWriteConfig cfg) {
|
||||
return new HoodieCleanClient(jsc, cfg, new HoodieMetrics(cfg, cfg.getTableName()));
|
||||
}
|
||||
|
||||
protected HoodieWriteClient getHoodieWriteClient(HoodieWriteConfig cfg) {
|
||||
return getHoodieWriteClient(cfg, false);
|
||||
}
|
||||
|
||||
69
hudi-common/src/main/avro/HoodieCleanerPlan.avsc
Normal file
69
hudi-common/src/main/avro/HoodieCleanerPlan.avsc
Normal file
@@ -0,0 +1,69 @@
|
||||
/*
|
||||
* 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.
|
||||
*/
|
||||
{
|
||||
"namespace": "org.apache.hudi.avro.model",
|
||||
"type": "record",
|
||||
"name": "HoodieCleanerPlan",
|
||||
"fields": [
|
||||
{
|
||||
"name": "earliestInstantToRetain",
|
||||
"type":["null", {
|
||||
"type": "record",
|
||||
"name": "HoodieActionInstant",
|
||||
"fields": [
|
||||
{
|
||||
"name": "timestamp",
|
||||
"type": "string"
|
||||
},
|
||||
{
|
||||
"name": "action",
|
||||
"type": "string"
|
||||
},
|
||||
{
|
||||
"name": "state",
|
||||
"type": "string"
|
||||
}
|
||||
]
|
||||
}],
|
||||
"default" : null
|
||||
},
|
||||
{
|
||||
"name": "policy",
|
||||
"type": "string"
|
||||
},
|
||||
{
|
||||
"name": "filesToBeDeletedPerPartition",
|
||||
"type": [
|
||||
"null", {
|
||||
"type":"map",
|
||||
"values": {
|
||||
"type":"array",
|
||||
"items":{
|
||||
"name":"filePath",
|
||||
"type": "string"
|
||||
}
|
||||
}}],
|
||||
"default" : null
|
||||
},
|
||||
{
|
||||
"name":"version",
|
||||
"type":["int", "null"],
|
||||
"default": 1
|
||||
}
|
||||
]
|
||||
}
|
||||
@@ -134,14 +134,9 @@ public class CompactionOperation implements Serializable {
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "CompactionOperation{"
|
||||
+ "baseInstantTime='" + baseInstantTime + '\''
|
||||
+ ", dataFileCommitTime=" + dataFileCommitTime
|
||||
+ ", deltaFileNames=" + deltaFileNames
|
||||
+ ", dataFileName=" + dataFileName
|
||||
+ ", id='" + id + '\''
|
||||
+ ", metrics=" + metrics
|
||||
+ '}';
|
||||
return "CompactionOperation{" + "baseInstantTime='" + baseInstantTime + '\'' + ", dataFileCommitTime="
|
||||
+ dataFileCommitTime + ", deltaFileNames=" + deltaFileNames + ", dataFileName=" + dataFileName + ", id='" + id
|
||||
+ '\'' + ", metrics=" + metrics + '}';
|
||||
}
|
||||
|
||||
@Override
|
||||
@@ -156,8 +151,7 @@ public class CompactionOperation implements Serializable {
|
||||
return Objects.equals(baseInstantTime, operation.baseInstantTime)
|
||||
&& Objects.equals(dataFileCommitTime, operation.dataFileCommitTime)
|
||||
&& Objects.equals(deltaFileNames, operation.deltaFileNames)
|
||||
&& Objects.equals(dataFileName, operation.dataFileName)
|
||||
&& Objects.equals(id, operation.id);
|
||||
&& Objects.equals(dataFileName, operation.dataFileName) && Objects.equals(id, operation.id);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
||||
@@ -63,6 +63,7 @@ public interface HoodieTimeline extends Serializable {
|
||||
String INFLIGHT_COMMIT_EXTENSION = INFLIGHT_EXTENSION;
|
||||
String INFLIGHT_DELTA_COMMIT_EXTENSION = "." + DELTA_COMMIT_ACTION + INFLIGHT_EXTENSION;
|
||||
String INFLIGHT_CLEAN_EXTENSION = "." + CLEAN_ACTION + INFLIGHT_EXTENSION;
|
||||
String REQUESTED_CLEAN_EXTENSION = "." + CLEAN_ACTION + REQUESTED_EXTENSION;
|
||||
String INFLIGHT_ROLLBACK_EXTENSION = "." + ROLLBACK_ACTION + INFLIGHT_EXTENSION;
|
||||
String INFLIGHT_SAVEPOINT_EXTENSION = "." + SAVEPOINT_ACTION + INFLIGHT_EXTENSION;
|
||||
String REQUESTED_COMPACTION_SUFFIX = StringUtils.join(COMPACTION_ACTION, REQUESTED_EXTENSION);
|
||||
@@ -80,6 +81,13 @@ public interface HoodieTimeline extends Serializable {
|
||||
*/
|
||||
HoodieTimeline filterInflights();
|
||||
|
||||
/**
|
||||
* Filter this timeline to include requested and in-flights
|
||||
*
|
||||
* @return New instance of HoodieTimeline with just in-flights and requested instants
|
||||
*/
|
||||
HoodieTimeline filterInflightsAndRequested();
|
||||
|
||||
/**
|
||||
* Filter this timeline to just include the in-flights excluding compaction instants
|
||||
*
|
||||
@@ -219,7 +227,19 @@ public interface HoodieTimeline extends Serializable {
|
||||
}
|
||||
|
||||
static HoodieInstant getCompletedInstant(final HoodieInstant instant) {
|
||||
return new HoodieInstant(false, instant.getAction(), instant.getTimestamp());
|
||||
return new HoodieInstant(State.COMPLETED, instant.getAction(), instant.getTimestamp());
|
||||
}
|
||||
|
||||
static HoodieInstant getRequestedInstant(final HoodieInstant instant) {
|
||||
return new HoodieInstant(State.REQUESTED, instant.getAction(), instant.getTimestamp());
|
||||
}
|
||||
|
||||
static HoodieInstant getCleanRequestedInstant(final String timestamp) {
|
||||
return new HoodieInstant(State.REQUESTED, CLEAN_ACTION, timestamp);
|
||||
}
|
||||
|
||||
static HoodieInstant getCleanInflightInstant(final String timestamp) {
|
||||
return new HoodieInstant(State.INFLIGHT, CLEAN_ACTION, timestamp);
|
||||
}
|
||||
|
||||
static HoodieInstant getCompactionRequestedInstant(final String timestamp) {
|
||||
@@ -246,6 +266,10 @@ public interface HoodieTimeline extends Serializable {
|
||||
return StringUtils.join(instant, HoodieTimeline.CLEAN_EXTENSION);
|
||||
}
|
||||
|
||||
static String makeRequestedCleanerFileName(String instant) {
|
||||
return StringUtils.join(instant, HoodieTimeline.REQUESTED_CLEAN_EXTENSION);
|
||||
}
|
||||
|
||||
static String makeInflightCleanerFileName(String instant) {
|
||||
return StringUtils.join(instant, HoodieTimeline.INFLIGHT_CLEAN_EXTENSION);
|
||||
}
|
||||
|
||||
@@ -58,10 +58,11 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
|
||||
|
||||
public static final SimpleDateFormat COMMIT_FORMATTER = new SimpleDateFormat("yyyyMMddHHmmss");
|
||||
|
||||
public static final Set<String> VALID_EXTENSIONS_IN_ACTIVE_TIMELINE = new HashSet<>(Arrays.asList(new String[] {
|
||||
COMMIT_EXTENSION, INFLIGHT_COMMIT_EXTENSION, DELTA_COMMIT_EXTENSION, INFLIGHT_DELTA_COMMIT_EXTENSION,
|
||||
SAVEPOINT_EXTENSION, INFLIGHT_SAVEPOINT_EXTENSION, CLEAN_EXTENSION, INFLIGHT_CLEAN_EXTENSION,
|
||||
INFLIGHT_COMPACTION_EXTENSION, REQUESTED_COMPACTION_EXTENSION, INFLIGHT_RESTORE_EXTENSION, RESTORE_EXTENSION}));
|
||||
public static final Set<String> VALID_EXTENSIONS_IN_ACTIVE_TIMELINE =
|
||||
new HashSet<>(Arrays.asList(new String[] {COMMIT_EXTENSION, INFLIGHT_COMMIT_EXTENSION, DELTA_COMMIT_EXTENSION,
|
||||
INFLIGHT_DELTA_COMMIT_EXTENSION, SAVEPOINT_EXTENSION, INFLIGHT_SAVEPOINT_EXTENSION, CLEAN_EXTENSION,
|
||||
INFLIGHT_CLEAN_EXTENSION, REQUESTED_CLEAN_EXTENSION, INFLIGHT_COMPACTION_EXTENSION,
|
||||
REQUESTED_COMPACTION_EXTENSION, INFLIGHT_RESTORE_EXTENSION, RESTORE_EXTENSION}));
|
||||
|
||||
private static final transient Logger log = LogManager.getLogger(HoodieActiveTimeline.class);
|
||||
protected HoodieTableMetaClient metaClient;
|
||||
@@ -212,11 +213,19 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
|
||||
}
|
||||
|
||||
public void revertToInflight(HoodieInstant instant) {
|
||||
log.info("Reverting instant to inflight " + instant);
|
||||
revertCompleteToInflight(instant, HoodieTimeline.getInflightInstant(instant));
|
||||
log.info("Reverting " + instant + " to inflight ");
|
||||
revertStateTransition(instant, HoodieTimeline.getInflightInstant(instant));
|
||||
log.info("Reverted " + instant + " to inflight");
|
||||
}
|
||||
|
||||
public HoodieInstant revertToRequested(HoodieInstant instant) {
|
||||
log.warn("Reverting " + instant + " to requested ");
|
||||
HoodieInstant requestedInstant = HoodieTimeline.getRequestedInstant(instant);
|
||||
revertStateTransition(instant, HoodieTimeline.getRequestedInstant(instant));
|
||||
log.warn("Reverted " + instant + " to requested");
|
||||
return requestedInstant;
|
||||
}
|
||||
|
||||
public void deleteInflight(HoodieInstant instant) {
|
||||
Preconditions.checkArgument(instant.isInflight());
|
||||
deleteInstantFile(instant);
|
||||
@@ -311,6 +320,39 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
|
||||
* END - COMPACTION RELATED META-DATA MANAGEMENT
|
||||
**/
|
||||
|
||||
/**
|
||||
* Transition Clean State from inflight to Committed
|
||||
*
|
||||
* @param inflightInstant Inflight instant
|
||||
* @param data Extra Metadata
|
||||
* @return commit instant
|
||||
*/
|
||||
public HoodieInstant transitionCleanInflightToComplete(HoodieInstant inflightInstant, Option<byte[]> data) {
|
||||
Preconditions.checkArgument(inflightInstant.getAction().equals(HoodieTimeline.CLEAN_ACTION));
|
||||
Preconditions.checkArgument(inflightInstant.isInflight());
|
||||
HoodieInstant commitInstant = new HoodieInstant(State.COMPLETED, CLEAN_ACTION, inflightInstant.getTimestamp());
|
||||
// First write metadata to aux folder
|
||||
createFileInAuxiliaryFolder(commitInstant, data);
|
||||
// Then write to timeline
|
||||
transitionState(inflightInstant, commitInstant, data);
|
||||
return commitInstant;
|
||||
}
|
||||
|
||||
/**
|
||||
* Transition Clean State from requested to inflight
|
||||
*
|
||||
* @param requestedInstant requested instant
|
||||
* @return commit instant
|
||||
*/
|
||||
public HoodieInstant transitionCleanRequestedToInflight(HoodieInstant requestedInstant) {
|
||||
Preconditions.checkArgument(requestedInstant.getAction().equals(HoodieTimeline.CLEAN_ACTION));
|
||||
Preconditions.checkArgument(requestedInstant.isRequested());
|
||||
HoodieInstant inflight = new HoodieInstant(State.INFLIGHT, CLEAN_ACTION, requestedInstant.getTimestamp());
|
||||
transitionState(requestedInstant, inflight, Option.empty());
|
||||
return inflight;
|
||||
}
|
||||
|
||||
|
||||
private void transitionState(HoodieInstant fromInstant, HoodieInstant toInstant, Option<byte[]> data) {
|
||||
Preconditions.checkArgument(fromInstant.getTimestamp().equals(toInstant.getTimestamp()));
|
||||
Path commitFilePath = new Path(metaClient.getMetaPath(), toInstant.getFileName());
|
||||
@@ -327,19 +369,20 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
|
||||
}
|
||||
}
|
||||
|
||||
private void revertCompleteToInflight(HoodieInstant completed, HoodieInstant inflight) {
|
||||
Preconditions.checkArgument(completed.getTimestamp().equals(inflight.getTimestamp()));
|
||||
Path inFlightCommitFilePath = new Path(metaClient.getMetaPath(), inflight.getFileName());
|
||||
private void revertStateTransition(HoodieInstant curr, HoodieInstant revert) {
|
||||
Preconditions.checkArgument(curr.getTimestamp().equals(revert.getTimestamp()));
|
||||
Path revertFilePath = new Path(metaClient.getMetaPath(), revert.getFileName());
|
||||
try {
|
||||
if (!metaClient.getFs().exists(inFlightCommitFilePath)) {
|
||||
Path commitFilePath = new Path(metaClient.getMetaPath(), completed.getFileName());
|
||||
boolean success = metaClient.getFs().rename(commitFilePath, inFlightCommitFilePath);
|
||||
if (!metaClient.getFs().exists(revertFilePath)) {
|
||||
Path currFilePath = new Path(metaClient.getMetaPath(), curr.getFileName());
|
||||
boolean success = metaClient.getFs().rename(currFilePath, revertFilePath);
|
||||
if (!success) {
|
||||
throw new HoodieIOException("Could not rename " + commitFilePath + " to " + inFlightCommitFilePath);
|
||||
throw new HoodieIOException("Could not rename " + currFilePath + " to " + revertFilePath);
|
||||
}
|
||||
log.info("Renamed " + currFilePath + " to " + revertFilePath);
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Could not complete revert " + completed, e);
|
||||
throw new HoodieIOException("Could not complete revert " + curr, e);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -355,6 +398,15 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
|
||||
createFileInMetaPath(instant.getFileName(), content);
|
||||
}
|
||||
|
||||
public void saveToCleanRequested(HoodieInstant instant, Option<byte[]> content) {
|
||||
Preconditions.checkArgument(instant.getAction().equals(HoodieTimeline.CLEAN_ACTION));
|
||||
Preconditions.checkArgument(instant.getState().equals(State.REQUESTED));
|
||||
// Write workload to auxiliary folder
|
||||
createFileInAuxiliaryFolder(instant, content);
|
||||
// Plan is only stored in auxiliary folder
|
||||
createFileInMetaPath(instant.getFileName(), Option.empty());
|
||||
}
|
||||
|
||||
private void createFileInMetaPath(String filename, Option<byte[]> content) {
|
||||
Path fullPath = new Path(metaClient.getMetaPath(), filename);
|
||||
createFileInPath(fullPath, content);
|
||||
|
||||
@@ -30,6 +30,7 @@ import java.util.function.Predicate;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
import org.apache.hudi.common.table.HoodieTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant.State;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.StringUtils;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
@@ -83,6 +84,13 @@ public class HoodieDefaultTimeline implements HoodieTimeline {
|
||||
return new HoodieDefaultTimeline(instants.stream().filter(HoodieInstant::isInflight), details);
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieTimeline filterInflightsAndRequested() {
|
||||
return new HoodieDefaultTimeline(
|
||||
instants.stream().filter(i -> i.getState().equals(State.REQUESTED) || i.getState().equals(State.INFLIGHT)),
|
||||
details);
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieTimeline filterInflightsExcludingCompaction() {
|
||||
return new HoodieDefaultTimeline(instants.stream().filter(instant -> {
|
||||
|
||||
@@ -69,7 +69,7 @@ public class HoodieInstant implements Serializable {
|
||||
} else if (action.contains(HoodieTimeline.INFLIGHT_EXTENSION)) {
|
||||
state = State.INFLIGHT;
|
||||
action = action.replace(HoodieTimeline.INFLIGHT_EXTENSION, "");
|
||||
} else if (action.equals(HoodieTimeline.REQUESTED_COMPACTION_SUFFIX)) {
|
||||
} else if (action.contains(HoodieTimeline.REQUESTED_EXTENSION)) {
|
||||
state = State.REQUESTED;
|
||||
action = action.replace(HoodieTimeline.REQUESTED_EXTENSION, "");
|
||||
}
|
||||
@@ -117,6 +117,7 @@ public class HoodieInstant implements Serializable {
|
||||
: HoodieTimeline.makeCommitFileName(timestamp);
|
||||
} else if (HoodieTimeline.CLEAN_ACTION.equals(action)) {
|
||||
return isInflight() ? HoodieTimeline.makeInflightCleanerFileName(timestamp)
|
||||
: isRequested() ? HoodieTimeline.makeRequestedCleanerFileName(timestamp)
|
||||
: HoodieTimeline.makeCleanerFileName(timestamp);
|
||||
} else if (HoodieTimeline.ROLLBACK_ACTION.equals(action)) {
|
||||
return isInflight() ? HoodieTimeline.makeInflightRollbackFileName(timestamp)
|
||||
|
||||
@@ -333,8 +333,10 @@ public class RocksDbBasedFileSystemView extends IncrementalTimelineSyncFileSyste
|
||||
|
||||
@Override
|
||||
public void close() {
|
||||
log.info("Closing Rocksdb !!");
|
||||
closed = true;
|
||||
rocksDB.close();
|
||||
log.info("Closed Rocksdb !!");
|
||||
}
|
||||
|
||||
@Override
|
||||
|
||||
@@ -36,6 +36,7 @@ import org.apache.avro.specific.SpecificDatumWriter;
|
||||
import org.apache.avro.specific.SpecificRecordBase;
|
||||
import org.apache.hudi.avro.model.HoodieCleanMetadata;
|
||||
import org.apache.hudi.avro.model.HoodieCleanPartitionMetadata;
|
||||
import org.apache.hudi.avro.model.HoodieCleanerPlan;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionPlan;
|
||||
import org.apache.hudi.avro.model.HoodieRestoreMetadata;
|
||||
import org.apache.hudi.avro.model.HoodieRollbackMetadata;
|
||||
@@ -110,6 +111,11 @@ public class AvroUtils {
|
||||
return serializeAvroMetadata(compactionWorkload, HoodieCompactionPlan.class);
|
||||
}
|
||||
|
||||
|
||||
public static Option<byte[]> serializeCleanerPlan(HoodieCleanerPlan cleanPlan) throws IOException {
|
||||
return serializeAvroMetadata(cleanPlan, HoodieCleanerPlan.class);
|
||||
}
|
||||
|
||||
public static Option<byte[]> serializeCleanMetadata(HoodieCleanMetadata metadata) throws IOException {
|
||||
return serializeAvroMetadata(metadata, HoodieCleanMetadata.class);
|
||||
}
|
||||
@@ -137,6 +143,10 @@ public class AvroUtils {
|
||||
return Option.of(baos.toByteArray());
|
||||
}
|
||||
|
||||
public static HoodieCleanerPlan deserializeCleanerPlan(byte[] bytes) throws IOException {
|
||||
return deserializeAvroMetadata(bytes, HoodieCleanerPlan.class);
|
||||
}
|
||||
|
||||
public static HoodieCompactionPlan deserializeCompactionPlan(byte[] bytes) throws IOException {
|
||||
return deserializeAvroMetadata(bytes, HoodieCompactionPlan.class);
|
||||
}
|
||||
|
||||
@@ -141,8 +141,7 @@ public class ParquetUtils {
|
||||
* Read out the bloom filter from the parquet file meta data.
|
||||
*/
|
||||
public static BloomFilter readBloomFilterFromParquetMetadata(Configuration configuration, Path parquetFilePath) {
|
||||
Map<String, String> footerVals =
|
||||
readParquetFooter(configuration, false, parquetFilePath,
|
||||
Map<String, String> footerVals = readParquetFooter(configuration, false, parquetFilePath,
|
||||
HoodieAvroWriteSupport.HOODIE_AVRO_BLOOM_FILTER_METADATA_KEY,
|
||||
HoodieAvroWriteSupport.OLD_HOODIE_AVRO_BLOOM_FILTER_METADATA_KEY);
|
||||
String footerVal = footerVals.get(HoodieAvroWriteSupport.HOODIE_AVRO_BLOOM_FILTER_METADATA_KEY);
|
||||
|
||||
@@ -27,6 +27,7 @@ import org.apache.hudi.common.util.collection.Pair;
|
||||
|
||||
/**
|
||||
* Migrates a specific metadata type stored in .hoodie folder to latest version
|
||||
*
|
||||
* @param <T>
|
||||
*/
|
||||
public class MetadataMigrator<T> {
|
||||
@@ -36,14 +37,15 @@ public class MetadataMigrator<T> {
|
||||
private final Integer oldestVersion;
|
||||
|
||||
public MetadataMigrator(HoodieTableMetaClient metaClient, List<VersionMigrator<T>> migratorList) {
|
||||
migrators = migratorList.stream().map(m ->
|
||||
Pair.of(m.getManagedVersion(), m)).collect(Collectors.toMap(Pair::getKey, Pair::getValue));
|
||||
migrators = migratorList.stream().map(m -> Pair.of(m.getManagedVersion(), m))
|
||||
.collect(Collectors.toMap(Pair::getKey, Pair::getValue));
|
||||
latestVersion = migrators.keySet().stream().reduce((x, y) -> x > y ? x : y).get();
|
||||
oldestVersion = migrators.keySet().stream().reduce((x, y) -> x < y ? x : y).get();
|
||||
}
|
||||
|
||||
/**
|
||||
* Upgrade Metadata version to its latest
|
||||
*
|
||||
* @param metadata Metadata
|
||||
* @param metadataVersion Current version of metadata
|
||||
* @return Metadata conforming to the latest version of this metadata
|
||||
@@ -64,6 +66,7 @@ public class MetadataMigrator<T> {
|
||||
|
||||
/**
|
||||
* Migrate metadata to a specific version
|
||||
*
|
||||
* @param metadata Hoodie Table Meta Client
|
||||
* @param metadataVersion Metadata Version
|
||||
* @param targetVersion Target Version
|
||||
|
||||
@@ -22,18 +22,21 @@ import java.io.Serializable;
|
||||
|
||||
/**
|
||||
* Responsible for upgrading and downgrading metadata versions for a specific metadata
|
||||
*
|
||||
* @param <T> Metadata Type
|
||||
*/
|
||||
public interface VersionMigrator<T> extends Serializable {
|
||||
|
||||
/**
|
||||
* Version of Metadata that this class will handle
|
||||
*
|
||||
* @return
|
||||
*/
|
||||
Integer getManagedVersion();
|
||||
|
||||
/**
|
||||
* Upgrades metadata of type T from previous version to this version
|
||||
*
|
||||
* @param input Metadata as of previous version.
|
||||
* @return Metadata compatible with the version managed by this class
|
||||
*/
|
||||
@@ -41,6 +44,7 @@ public interface VersionMigrator<T> extends Serializable {
|
||||
|
||||
/**
|
||||
* Downgrades metadata of type T from next version to this version
|
||||
*
|
||||
* @param input Metadata as of next highest version
|
||||
* @return Metadata compatible with the version managed by this class
|
||||
*/
|
||||
|
||||
@@ -29,8 +29,7 @@ import org.apache.hudi.common.versioning.MetadataMigrator;
|
||||
public class CompactionPlanMigrator extends MetadataMigrator<HoodieCompactionPlan> {
|
||||
|
||||
public CompactionPlanMigrator(HoodieTableMetaClient metaClient) {
|
||||
super(metaClient, Arrays.asList(
|
||||
new CompactionV1MigrationHandler(metaClient),
|
||||
new CompactionV2MigrationHandler(metaClient)));
|
||||
super(metaClient,
|
||||
Arrays.asList(new CompactionV1MigrationHandler(metaClient), new CompactionV2MigrationHandler(metaClient)));
|
||||
}
|
||||
}
|
||||
|
||||
@@ -52,21 +52,17 @@ public class CompactionV1MigrationHandler extends AbstractMigratorBase<HoodieCom
|
||||
|
||||
@Override
|
||||
public HoodieCompactionPlan downgradeFrom(HoodieCompactionPlan input) {
|
||||
Preconditions.checkArgument(input.getVersion() == 2, "Input version is "
|
||||
+ input.getVersion() + ". Must be 2");
|
||||
Preconditions.checkArgument(input.getVersion() == 2, "Input version is " + input.getVersion() + ". Must be 2");
|
||||
HoodieCompactionPlan compactionPlan = new HoodieCompactionPlan();
|
||||
final Path basePath = new Path(metaClient.getBasePath());
|
||||
List<HoodieCompactionOperation> v1CompactionOperationList = new ArrayList<>();
|
||||
if (null != input.getOperations()) {
|
||||
v1CompactionOperationList = input.getOperations().stream().map(inp -> {
|
||||
return HoodieCompactionOperation.newBuilder()
|
||||
.setBaseInstantTime(inp.getBaseInstantTime())
|
||||
.setFileId(inp.getFileId())
|
||||
.setPartitionPath(inp.getPartitionPath())
|
||||
.setMetrics(inp.getMetrics())
|
||||
return HoodieCompactionOperation.newBuilder().setBaseInstantTime(inp.getBaseInstantTime())
|
||||
.setFileId(inp.getFileId()).setPartitionPath(inp.getPartitionPath()).setMetrics(inp.getMetrics())
|
||||
.setDataFilePath(convertToV1Path(basePath, inp.getPartitionPath(), inp.getDataFilePath()))
|
||||
.setDeltaFilePaths(inp.getDeltaFilePaths().stream().map(s -> convertToV1Path(basePath,
|
||||
inp.getPartitionPath(), s)).collect(Collectors.toList()))
|
||||
.setDeltaFilePaths(inp.getDeltaFilePaths().stream()
|
||||
.map(s -> convertToV1Path(basePath, inp.getPartitionPath(), s)).collect(Collectors.toList()))
|
||||
.build();
|
||||
}).collect(Collectors.toList());
|
||||
}
|
||||
|
||||
@@ -46,20 +46,15 @@ public class CompactionV2MigrationHandler extends AbstractMigratorBase<HoodieCom
|
||||
|
||||
@Override
|
||||
public HoodieCompactionPlan upgradeFrom(HoodieCompactionPlan input) {
|
||||
Preconditions.checkArgument(input.getVersion() == 1, "Input version is "
|
||||
+ input.getVersion() + ". Must be 1");
|
||||
Preconditions.checkArgument(input.getVersion() == 1, "Input version is " + input.getVersion() + ". Must be 1");
|
||||
HoodieCompactionPlan compactionPlan = new HoodieCompactionPlan();
|
||||
List<HoodieCompactionOperation> v2CompactionOperationList = new ArrayList<>();
|
||||
if (null != input.getOperations()) {
|
||||
v2CompactionOperationList = input.getOperations().stream().map(inp -> {
|
||||
return HoodieCompactionOperation.newBuilder()
|
||||
.setBaseInstantTime(inp.getBaseInstantTime())
|
||||
.setFileId(inp.getFileId())
|
||||
.setPartitionPath(inp.getPartitionPath())
|
||||
.setMetrics(inp.getMetrics())
|
||||
.setDataFilePath(new Path(inp.getDataFilePath()).getName())
|
||||
.setDeltaFilePaths(inp.getDeltaFilePaths().stream().map(s -> new Path(s).getName())
|
||||
.collect(Collectors.toList()))
|
||||
return HoodieCompactionOperation.newBuilder().setBaseInstantTime(inp.getBaseInstantTime())
|
||||
.setFileId(inp.getFileId()).setPartitionPath(inp.getPartitionPath()).setMetrics(inp.getMetrics())
|
||||
.setDataFilePath(new Path(inp.getDataFilePath()).getName()).setDeltaFilePaths(
|
||||
inp.getDeltaFilePaths().stream().map(s -> new Path(s).getName()).collect(Collectors.toList()))
|
||||
.build();
|
||||
}).collect(Collectors.toList());
|
||||
}
|
||||
|
||||
@@ -114,8 +114,7 @@ public class TestCompactionUtils extends HoodieCommonTestHarness {
|
||||
fileSlice.addLogFile(
|
||||
new HoodieLogFile(new Path(FSUtils.makeLogFileName("noData1", ".log", "000", 2, TEST_WRITE_TOKEN))));
|
||||
op = CompactionUtils.buildFromFileSlice(DEFAULT_PARTITION_PATHS[0], fileSlice, Option.of(metricsCaptureFn));
|
||||
testFileSliceCompactionOpEquality(fileSlice, op, DEFAULT_PARTITION_PATHS[0],
|
||||
LATEST_COMPACTION_METADATA_VERSION);
|
||||
testFileSliceCompactionOpEquality(fileSlice, op, DEFAULT_PARTITION_PATHS[0], LATEST_COMPACTION_METADATA_VERSION);
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -126,17 +125,17 @@ public class TestCompactionUtils extends HoodieCommonTestHarness {
|
||||
FileSlice emptyFileSlice = new FileSlice(DEFAULT_PARTITION_PATHS[0], "000", "empty1");
|
||||
FileSlice fileSlice = new FileSlice(DEFAULT_PARTITION_PATHS[0], "000", "noData1");
|
||||
fileSlice.setDataFile(new TestHoodieDataFile(fullPartitionPath.toString() + "/data1_1_000.parquet"));
|
||||
fileSlice.addLogFile(
|
||||
new HoodieLogFile(new Path(fullPartitionPath, new Path(FSUtils.makeLogFileName("noData1", ".log", "000", 1, TEST_WRITE_TOKEN)))));
|
||||
fileSlice.addLogFile(
|
||||
new HoodieLogFile(new Path(fullPartitionPath, new Path(FSUtils.makeLogFileName("noData1", ".log", "000", 2, TEST_WRITE_TOKEN)))));
|
||||
fileSlice.addLogFile(new HoodieLogFile(
|
||||
new Path(fullPartitionPath, new Path(FSUtils.makeLogFileName("noData1", ".log", "000", 1, TEST_WRITE_TOKEN)))));
|
||||
fileSlice.addLogFile(new HoodieLogFile(
|
||||
new Path(fullPartitionPath, new Path(FSUtils.makeLogFileName("noData1", ".log", "000", 2, TEST_WRITE_TOKEN)))));
|
||||
FileSlice noLogFileSlice = new FileSlice(DEFAULT_PARTITION_PATHS[0], "000", "noLog1");
|
||||
noLogFileSlice.setDataFile(new TestHoodieDataFile(fullPartitionPath.toString() + "/noLog_1_000.parquet"));
|
||||
FileSlice noDataFileSlice = new FileSlice(DEFAULT_PARTITION_PATHS[0], "000", "noData1");
|
||||
noDataFileSlice.addLogFile(
|
||||
new HoodieLogFile(new Path(fullPartitionPath, new Path(FSUtils.makeLogFileName("noData1", ".log", "000", 1, TEST_WRITE_TOKEN)))));
|
||||
noDataFileSlice.addLogFile(
|
||||
new HoodieLogFile(new Path(fullPartitionPath, new Path(FSUtils.makeLogFileName("noData1", ".log", "000", 2, TEST_WRITE_TOKEN)))));
|
||||
noDataFileSlice.addLogFile(new HoodieLogFile(
|
||||
new Path(fullPartitionPath, new Path(FSUtils.makeLogFileName("noData1", ".log", "000", 1, TEST_WRITE_TOKEN)))));
|
||||
noDataFileSlice.addLogFile(new HoodieLogFile(
|
||||
new Path(fullPartitionPath, new Path(FSUtils.makeLogFileName("noData1", ".log", "000", 2, TEST_WRITE_TOKEN)))));
|
||||
List<FileSlice> fileSliceList = Arrays.asList(emptyFileSlice, noDataFileSlice, fileSlice, noLogFileSlice);
|
||||
List<Pair<String, FileSlice>> input =
|
||||
fileSliceList.stream().map(f -> Pair.of(DEFAULT_PARTITION_PATHS[0], f)).collect(Collectors.toList());
|
||||
@@ -221,9 +220,8 @@ public class TestCompactionUtils extends HoodieCommonTestHarness {
|
||||
*/
|
||||
private void testFileSlicesCompactionPlanEquality(List<Pair<String, FileSlice>> input, HoodieCompactionPlan plan) {
|
||||
Assert.assertEquals("All file-slices present", input.size(), plan.getOperations().size());
|
||||
IntStream.range(0, input.size()).boxed().forEach(idx ->
|
||||
testFileSliceCompactionOpEquality(input.get(idx).getValue(), plan.getOperations().get(idx),
|
||||
input.get(idx).getKey(), plan.getVersion()));
|
||||
IntStream.range(0, input.size()).boxed().forEach(idx -> testFileSliceCompactionOpEquality(input.get(idx).getValue(),
|
||||
plan.getOperations().get(idx), input.get(idx).getKey(), plan.getVersion()));
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -233,15 +231,15 @@ public class TestCompactionUtils extends HoodieCommonTestHarness {
|
||||
* @param op HoodieCompactionOperation
|
||||
* @param expPartitionPath Partition path
|
||||
*/
|
||||
private void testFileSliceCompactionOpEquality(FileSlice slice, HoodieCompactionOperation op,
|
||||
String expPartitionPath, int version) {
|
||||
private void testFileSliceCompactionOpEquality(FileSlice slice, HoodieCompactionOperation op, String expPartitionPath,
|
||||
int version) {
|
||||
Assert.assertEquals("Partition path is correct", expPartitionPath, op.getPartitionPath());
|
||||
Assert.assertEquals("Same base-instant", slice.getBaseInstantTime(), op.getBaseInstantTime());
|
||||
Assert.assertEquals("Same file-id", slice.getFileId(), op.getFileId());
|
||||
if (slice.getDataFile().isPresent()) {
|
||||
HoodieDataFile df = slice.getDataFile().get();
|
||||
Assert.assertEquals("Same data-file",
|
||||
version == COMPACTION_METADATA_VERSION_1 ? df.getPath() : df.getFileName(), op.getDataFilePath());
|
||||
Assert.assertEquals("Same data-file", version == COMPACTION_METADATA_VERSION_1 ? df.getPath() : df.getFileName(),
|
||||
op.getDataFilePath());
|
||||
}
|
||||
List<String> paths = slice.getLogFiles().map(l -> l.getPath().toString()).collect(Collectors.toList());
|
||||
IntStream.range(0, paths.size()).boxed().forEach(idx -> {
|
||||
|
||||
@@ -62,6 +62,7 @@ public class HiveSyncTool {
|
||||
}
|
||||
|
||||
public void syncHoodieTable() throws ClassNotFoundException {
|
||||
try {
|
||||
switch (hoodieHiveClient.getTableType()) {
|
||||
case COPY_ON_WRITE:
|
||||
syncHoodieTable(false);
|
||||
@@ -80,8 +81,12 @@ public class HiveSyncTool {
|
||||
LOG.error("Unknown table type " + hoodieHiveClient.getTableType());
|
||||
throw new InvalidDatasetException(hoodieHiveClient.getBasePath());
|
||||
}
|
||||
} catch (RuntimeException re) {
|
||||
LOG.error("Got runtime exception when hive syncing", re);
|
||||
} finally {
|
||||
hoodieHiveClient.close();
|
||||
}
|
||||
}
|
||||
|
||||
private void syncHoodieTable(boolean isRealTime) throws ClassNotFoundException {
|
||||
LOG.info("Trying to sync hoodie table " + cfg.tableName + " with base path " + hoodieHiveClient.getBasePath()
|
||||
|
||||
@@ -138,9 +138,11 @@ public class TimelineService {
|
||||
}
|
||||
|
||||
public void close() {
|
||||
log.info("Closing Timeline Service");
|
||||
this.app.stop();
|
||||
this.app = null;
|
||||
this.fsViewsManager.close();
|
||||
log.info("Closed Timeline Service");
|
||||
}
|
||||
|
||||
public Configuration getConf() {
|
||||
|
||||
Reference in New Issue
Block a user