Track fileIds with pending compaction in FileSystemView to provide correct API semantics
This commit is contained in:
committed by
vinoth chandar
parent
1b61f04e05
commit
9d99942564
@@ -16,6 +16,7 @@
|
||||
|
||||
package com.uber.hoodie.common.table.view;
|
||||
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.uber.hoodie.common.model.FileSlice;
|
||||
import com.uber.hoodie.common.model.HoodieDataFile;
|
||||
import com.uber.hoodie.common.model.HoodieFileGroup;
|
||||
@@ -23,6 +24,8 @@ import com.uber.hoodie.common.model.HoodieLogFile;
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.common.table.HoodieTimeline;
|
||||
import com.uber.hoodie.common.table.TableFileSystemView;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieInstant;
|
||||
import com.uber.hoodie.common.util.CompactionUtils;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import com.uber.hoodie.exception.HoodieIOException;
|
||||
import java.io.IOException;
|
||||
@@ -41,6 +44,8 @@ import java.util.stream.Stream;
|
||||
import org.apache.commons.lang3.tuple.Pair;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
/**
|
||||
* Common abstract implementation for multiple TableFileSystemView Implementations. 2 possible
|
||||
@@ -55,6 +60,8 @@ public class HoodieTableFileSystemView implements TableFileSystemView,
|
||||
TableFileSystemView.ReadOptimizedView,
|
||||
TableFileSystemView.RealtimeView, Serializable {
|
||||
|
||||
private static Logger log = LogManager.getLogger(HoodieTableFileSystemView.class);
|
||||
|
||||
protected HoodieTableMetaClient metaClient;
|
||||
// This is the commits that will be visible for all views extending this view
|
||||
protected HoodieTimeline visibleActiveTimeline;
|
||||
@@ -78,10 +85,15 @@ public class HoodieTableFileSystemView implements TableFileSystemView,
|
||||
this.visibleActiveTimeline = visibleActiveTimeline;
|
||||
this.fileGroupMap = new HashMap<>();
|
||||
this.partitionToFileGroupsMap = new HashMap<>();
|
||||
//TODO: vb Will be implemented in next PR
|
||||
this.fileIdToPendingCompactionInstantTime = new HashMap<>();
|
||||
}
|
||||
|
||||
// Build fileId to Pending Compaction Instants
|
||||
List<HoodieInstant> pendingCompactionInstants =
|
||||
metaClient.getActiveTimeline().filterPendingCompactionTimeline().getInstants().collect(Collectors.toList());
|
||||
this.fileIdToPendingCompactionInstantTime = ImmutableMap.copyOf(
|
||||
CompactionUtils.getAllPendingCompactionOperations(metaClient).entrySet().stream().map(entry -> {
|
||||
return Pair.of(entry.getKey(), entry.getValue().getKey());
|
||||
}).collect(Collectors.toMap(Pair::getKey, Pair::getValue)));
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a file system view, as of the given timeline, with the provided file statuses.
|
||||
@@ -404,15 +416,4 @@ public class HoodieTableFileSystemView implements TableFileSystemView,
|
||||
"Failed to list data files in partition " + partitionPathStr, e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Used by tests to add pending compaction entries TODO: This method is temporary and should go away in subsequent
|
||||
* Async Compaction PR
|
||||
*
|
||||
* @param fileId File Id
|
||||
* @param compactionInstantTime Compaction Instant Time
|
||||
*/
|
||||
protected void addPendingCompactionFileId(String fileId, String compactionInstantTime) {
|
||||
fileIdToPendingCompactionInstantTime.put(fileId, compactionInstantTime);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -174,7 +174,7 @@ public class AvroUtils {
|
||||
return Optional.of(baos.toByteArray());
|
||||
}
|
||||
|
||||
public static HoodieCompactionPlan deserializeHoodieCompactionPlan(byte[] bytes)
|
||||
public static HoodieCompactionPlan deserializeCompactionPlan(byte[] bytes)
|
||||
throws IOException {
|
||||
return deserializeAvroMetadata(bytes, HoodieCompactionPlan.class);
|
||||
}
|
||||
|
||||
@@ -20,15 +20,21 @@ import com.uber.hoodie.avro.model.HoodieCompactionOperation;
|
||||
import com.uber.hoodie.avro.model.HoodieCompactionPlan;
|
||||
import com.uber.hoodie.common.model.CompactionOperation;
|
||||
import com.uber.hoodie.common.model.FileSlice;
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieInstant;
|
||||
import com.uber.hoodie.exception.HoodieException;
|
||||
import java.io.IOException;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.function.Function;
|
||||
import java.util.stream.Collectors;
|
||||
import javafx.util.Pair;
|
||||
import java.util.stream.Stream;
|
||||
import org.apache.commons.lang3.tuple.Pair;
|
||||
|
||||
/**
|
||||
* Helper class to generate compaction workload from FileGroup/FileSlice abstraction
|
||||
* Helper class to generate compaction plan from FileGroup/FileSlice abstraction
|
||||
*/
|
||||
public class CompactionUtils {
|
||||
|
||||
@@ -41,7 +47,7 @@ public class CompactionUtils {
|
||||
* @return Compaction Operation
|
||||
*/
|
||||
public static HoodieCompactionOperation buildFromFileSlice(String partitionPath, FileSlice fileSlice,
|
||||
Optional<Function<Pair<String, FileSlice>, Map<String, Long>>> metricsCaptureFunction) {
|
||||
Optional<Function<Pair<String, FileSlice>, Map<String, Double>>> metricsCaptureFunction) {
|
||||
HoodieCompactionOperation.Builder builder = HoodieCompactionOperation.newBuilder();
|
||||
builder.setPartitionPath(partitionPath);
|
||||
builder.setFileId(fileSlice.getFileId());
|
||||
@@ -52,25 +58,23 @@ public class CompactionUtils {
|
||||
}
|
||||
|
||||
if (metricsCaptureFunction.isPresent()) {
|
||||
builder.setMetrics(metricsCaptureFunction.get().apply(new Pair(partitionPath, fileSlice)));
|
||||
builder.setMetrics(metricsCaptureFunction.get().apply(Pair.of(partitionPath, fileSlice)));
|
||||
}
|
||||
return builder.build();
|
||||
}
|
||||
|
||||
/**
|
||||
* Generate compaction workload from file-slices
|
||||
* Generate compaction plan from file-slices
|
||||
*
|
||||
* @param compactorId Compactor Id to set
|
||||
* @param partitionFileSlicePairs list of partition file-slice pairs
|
||||
* @param extraMetadata Extra Metadata
|
||||
* @param metricsCaptureFunction Metrics Capture function
|
||||
*/
|
||||
public static HoodieCompactionPlan buildFromFileSlices(String compactorId,
|
||||
public static HoodieCompactionPlan buildFromFileSlices(
|
||||
List<Pair<String, FileSlice>> partitionFileSlicePairs,
|
||||
Optional<Map<String, String>> extraMetadata,
|
||||
Optional<Function<Pair<String, FileSlice>, Map<String, Long>>> metricsCaptureFunction) {
|
||||
Optional<Function<Pair<String, FileSlice>, Map<String, Double>>> metricsCaptureFunction) {
|
||||
HoodieCompactionPlan.Builder builder = HoodieCompactionPlan.newBuilder();
|
||||
builder.setCompactorId(compactorId);
|
||||
extraMetadata.ifPresent(m -> builder.setExtraMetadata(m));
|
||||
builder.setOperations(partitionFileSlicePairs.stream().map(pfPair ->
|
||||
buildFromFileSlice(pfPair.getKey(), pfPair.getValue(), metricsCaptureFunction)).collect(Collectors.toList()));
|
||||
@@ -97,4 +101,58 @@ public class CompactionUtils {
|
||||
public static CompactionOperation buildCompactionOperation(HoodieCompactionOperation hc) {
|
||||
return CompactionOperation.convertFromAvroRecordInstance(hc);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get all pending compaction plans along with their instants
|
||||
*
|
||||
* @param metaClient Hoodie Meta Client
|
||||
*/
|
||||
public static List<Pair<HoodieInstant, HoodieCompactionPlan>> getAllPendingCompactionPlans(
|
||||
HoodieTableMetaClient metaClient) {
|
||||
List<HoodieInstant> pendingCompactionInstants =
|
||||
metaClient.getActiveTimeline().filterPendingCompactionTimeline().getInstants().collect(Collectors.toList());
|
||||
return pendingCompactionInstants.stream().map(instant -> {
|
||||
try {
|
||||
HoodieCompactionPlan compactionPlan = AvroUtils.deserializeCompactionPlan(
|
||||
metaClient.getActiveTimeline().getInstantDetails(instant).get());
|
||||
return Pair.of(instant, compactionPlan);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieException(e);
|
||||
}
|
||||
}).collect(Collectors.toList());
|
||||
}
|
||||
|
||||
/**
|
||||
* Get all file-ids with pending Compaction operations and their target compaction instant time
|
||||
*
|
||||
* @param metaClient Hoodie Table Meta Client
|
||||
*/
|
||||
public static Map<String, Pair<String, HoodieCompactionOperation>> getAllPendingCompactionOperations(
|
||||
HoodieTableMetaClient metaClient) {
|
||||
List<Pair<HoodieInstant, HoodieCompactionPlan>> pendingCompactionPlanWithInstants =
|
||||
getAllPendingCompactionPlans(metaClient);
|
||||
|
||||
Map<String, Pair<String, HoodieCompactionOperation>> fileIdToPendingCompactionWithInstantMap = new HashMap<>();
|
||||
pendingCompactionPlanWithInstants.stream().flatMap(instantPlanPair -> {
|
||||
HoodieInstant instant = instantPlanPair.getKey();
|
||||
HoodieCompactionPlan compactionPlan = instantPlanPair.getValue();
|
||||
List<HoodieCompactionOperation> ops = compactionPlan.getOperations();
|
||||
if (null != ops) {
|
||||
return ops.stream().map(op -> {
|
||||
return Pair.of(op.getFileId(), Pair.of(instant.getTimestamp(), op));
|
||||
});
|
||||
} else {
|
||||
return Stream.empty();
|
||||
}
|
||||
}).forEach(pair -> {
|
||||
// Defensive check to ensure a single-fileId does not have more than one pending compaction
|
||||
if (fileIdToPendingCompactionWithInstantMap.containsKey(pair.getKey())) {
|
||||
String msg = "Hoodie File Id (" + pair.getKey() + ") has more thant 1 pending compactions. Instants: "
|
||||
+ pair.getValue() + ", " + fileIdToPendingCompactionWithInstantMap.get(pair.getKey());
|
||||
throw new IllegalStateException(msg);
|
||||
}
|
||||
fileIdToPendingCompactionWithInstantMap.put(pair.getKey(), pair.getValue());
|
||||
});
|
||||
return fileIdToPendingCompactionWithInstantMap;
|
||||
}
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user