[HUDI-308] Avoid Renames for tracking state transitions of all actions on dataset
This commit is contained in:
committed by
Balaji Varadarajan
parent
8963a68e6a
commit
9a1f698eef
@@ -18,15 +18,18 @@
|
||||
|
||||
package org.apache.hudi.cli;
|
||||
|
||||
import org.apache.hudi.common.model.TimelineLayoutVersion;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.util.ConsistencyGuardConfig;
|
||||
import org.apache.hudi.common.util.FSUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
|
||||
/**
|
||||
* This class is responsible to load table metadata and hoodie related configs.
|
||||
*/
|
||||
@@ -39,6 +42,7 @@ public class HoodieCLI {
|
||||
public static String basePath;
|
||||
public static HoodieTableMetaClient tableMetadata;
|
||||
public static HoodieTableMetaClient syncTableMetadata;
|
||||
public static TimelineLayoutVersion layoutVersion;
|
||||
|
||||
/**
|
||||
* Enum for CLI state.
|
||||
@@ -59,6 +63,11 @@ public class HoodieCLI {
|
||||
HoodieCLI.basePath = basePath;
|
||||
}
|
||||
|
||||
private static void setLayoutVersion(Integer layoutVersion) {
|
||||
HoodieCLI.layoutVersion = new TimelineLayoutVersion(
|
||||
(layoutVersion == null) ? TimelineLayoutVersion.CURR_VERSION : layoutVersion);
|
||||
}
|
||||
|
||||
public static boolean initConf() {
|
||||
if (HoodieCLI.conf == null) {
|
||||
HoodieCLI.conf = FSUtils.prepareHadoopConf(new Configuration());
|
||||
@@ -74,11 +83,13 @@ public class HoodieCLI {
|
||||
}
|
||||
|
||||
public static void refreshTableMetadata() {
|
||||
setTableMetaClient(new HoodieTableMetaClient(HoodieCLI.conf, basePath, false, HoodieCLI.consistencyGuardConfig));
|
||||
setTableMetaClient(new HoodieTableMetaClient(HoodieCLI.conf, basePath, false, HoodieCLI.consistencyGuardConfig,
|
||||
Option.of(layoutVersion)));
|
||||
}
|
||||
|
||||
public static void connectTo(String basePath) {
|
||||
public static void connectTo(String basePath, Integer layoutVersion) {
|
||||
setBasePath(basePath);
|
||||
setLayoutVersion(layoutVersion);
|
||||
refreshTableMetadata();
|
||||
}
|
||||
}
|
||||
|
||||
@@ -97,32 +97,33 @@ public class CompactionCommand implements CommandMarker {
|
||||
List<Comparable[]> rows = new ArrayList<>();
|
||||
for (int i = 0; i < instants.size(); i++) {
|
||||
HoodieInstant instant = instants.get(i);
|
||||
HoodieCompactionPlan workload = null;
|
||||
HoodieCompactionPlan compactionPlan = null;
|
||||
if (!instant.getAction().equals(HoodieTimeline.COMPACTION_ACTION)) {
|
||||
try {
|
||||
// This could be a completed compaction. Assume a compaction request file is present but skip if fails
|
||||
workload = AvroUtils.deserializeCompactionPlan(activeTimeline
|
||||
.getInstantAuxiliaryDetails(HoodieTimeline.getCompactionRequestedInstant(instant.getTimestamp())).get());
|
||||
compactionPlan = AvroUtils.deserializeCompactionPlan(
|
||||
activeTimeline.readPlanAsBytes(
|
||||
HoodieTimeline.getCompactionRequestedInstant(instant.getTimestamp())).get());
|
||||
} catch (HoodieIOException ioe) {
|
||||
// SKIP
|
||||
}
|
||||
} else {
|
||||
workload = AvroUtils.deserializeCompactionPlan(activeTimeline
|
||||
.getInstantAuxiliaryDetails(HoodieTimeline.getCompactionRequestedInstant(instant.getTimestamp())).get());
|
||||
compactionPlan = AvroUtils.deserializeCompactionPlan(activeTimeline.readPlanAsBytes(
|
||||
HoodieTimeline.getCompactionRequestedInstant(instant.getTimestamp())).get());
|
||||
}
|
||||
|
||||
if (null != workload) {
|
||||
if (null != compactionPlan) {
|
||||
HoodieInstant.State state = instant.getState();
|
||||
if (committed.contains(instant.getTimestamp())) {
|
||||
state = State.COMPLETED;
|
||||
}
|
||||
if (includeExtraMetadata) {
|
||||
rows.add(new Comparable[] {instant.getTimestamp(), state.toString(),
|
||||
workload.getOperations() == null ? 0 : workload.getOperations().size(),
|
||||
workload.getExtraMetadata().toString()});
|
||||
compactionPlan.getOperations() == null ? 0 : compactionPlan.getOperations().size(),
|
||||
compactionPlan.getExtraMetadata().toString()});
|
||||
} else {
|
||||
rows.add(new Comparable[] {instant.getTimestamp(), state.toString(),
|
||||
workload.getOperations() == null ? 0 : workload.getOperations().size()});
|
||||
compactionPlan.getOperations() == null ? 0 : compactionPlan.getOperations().size()});
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -148,12 +149,13 @@ public class CompactionCommand implements CommandMarker {
|
||||
unspecifiedDefaultValue = "false") final boolean headerOnly)
|
||||
throws Exception {
|
||||
HoodieActiveTimeline activeTimeline = HoodieCLI.tableMetadata.getActiveTimeline();
|
||||
HoodieCompactionPlan workload = AvroUtils.deserializeCompactionPlan(activeTimeline
|
||||
.getInstantAuxiliaryDetails(HoodieTimeline.getCompactionRequestedInstant(compactionInstantTime)).get());
|
||||
HoodieCompactionPlan compactionPlan = AvroUtils.deserializeCompactionPlan(
|
||||
activeTimeline.readPlanAsBytes(
|
||||
HoodieTimeline.getCompactionRequestedInstant(compactionInstantTime)).get());
|
||||
|
||||
List<Comparable[]> rows = new ArrayList<>();
|
||||
if ((null != workload) && (null != workload.getOperations())) {
|
||||
for (HoodieCompactionOperation op : workload.getOperations()) {
|
||||
if ((null != compactionPlan) && (null != compactionPlan.getOperations())) {
|
||||
for (HoodieCompactionOperation op : compactionPlan.getOperations()) {
|
||||
rows.add(new Comparable[] {op.getPartitionPath(), op.getFileId(), op.getBaseInstantTime(), op.getDataFilePath(),
|
||||
op.getDeltaFilePaths().size(), op.getMetrics() == null ? "" : op.getMetrics().toString()});
|
||||
}
|
||||
@@ -173,7 +175,7 @@ public class CompactionCommand implements CommandMarker {
|
||||
HoodieCLI.initFS(initialized);
|
||||
|
||||
// First get a compaction instant time and pass it to spark launcher for scheduling compaction
|
||||
String compactionInstantTime = HoodieActiveTimeline.createNewCommitTime();
|
||||
String compactionInstantTime = HoodieActiveTimeline.createNewInstantTime();
|
||||
|
||||
if (HoodieCLI.tableMetadata.getTableType() == HoodieTableType.MERGE_ON_READ) {
|
||||
String sparkPropertiesPath =
|
||||
|
||||
@@ -50,6 +50,7 @@ public class DatasetsCommand implements CommandMarker {
|
||||
@CliCommand(value = "connect", help = "Connect to a hoodie dataset")
|
||||
public String connect(
|
||||
@CliOption(key = {"path"}, mandatory = true, help = "Base Path of the dataset") final String path,
|
||||
@CliOption(key = {"layoutVersion"}, mandatory = false, help = "Timeline Layout version") Integer layoutVersion,
|
||||
@CliOption(key = {"eventuallyConsistent"}, mandatory = false, unspecifiedDefaultValue = "false",
|
||||
help = "Enable eventual consistency") final boolean eventuallyConsistent,
|
||||
@CliOption(key = {"initialCheckIntervalMs"}, mandatory = false, unspecifiedDefaultValue = "2000",
|
||||
@@ -65,7 +66,7 @@ public class DatasetsCommand implements CommandMarker {
|
||||
.withMaxConsistencyCheckIntervalMs(maxConsistencyIntervalMs).withMaxConsistencyChecks(maxConsistencyChecks)
|
||||
.build());
|
||||
HoodieCLI.initConf();
|
||||
HoodieCLI.connectTo(path);
|
||||
HoodieCLI.connectTo(path, layoutVersion);
|
||||
HoodieCLI.initFS(true);
|
||||
HoodieCLI.state = HoodieCLI.CLIState.DATASET;
|
||||
return "Metadata for table " + HoodieCLI.tableMetadata.getTableConfig().getTableName() + " loaded";
|
||||
@@ -85,6 +86,8 @@ public class DatasetsCommand implements CommandMarker {
|
||||
@CliOption(key = {"tableName"}, mandatory = true, help = "Hoodie Table Name") final String name,
|
||||
@CliOption(key = {"tableType"}, unspecifiedDefaultValue = "COPY_ON_WRITE",
|
||||
help = "Hoodie Table Type. Must be one of : COPY_ON_WRITE or MERGE_ON_READ") final String tableTypeStr,
|
||||
@CliOption(key = {"archiveLogFolder"}, help = "Folder Name for storing archived timeline") String archiveFolder,
|
||||
@CliOption(key = {"layoutVersion"}, help = "Specific Layout Version to use") Integer layoutVersion,
|
||||
@CliOption(key = {"payloadClass"}, unspecifiedDefaultValue = "org.apache.hudi.common.model.HoodieAvroPayload",
|
||||
help = "Payload Class") final String payloadClass)
|
||||
throws IOException {
|
||||
@@ -106,10 +109,11 @@ public class DatasetsCommand implements CommandMarker {
|
||||
}
|
||||
|
||||
final HoodieTableType tableType = HoodieTableType.valueOf(tableTypeStr);
|
||||
HoodieTableMetaClient.initTableType(HoodieCLI.conf, path, tableType, name, payloadClass);
|
||||
HoodieTableMetaClient.initTableType(HoodieCLI.conf, path, tableType, name, archiveFolder,
|
||||
payloadClass, layoutVersion);
|
||||
|
||||
// Now connect to ensure loading works
|
||||
return connect(path, false, 0, 0, 0);
|
||||
return connect(path, layoutVersion, false, 0, 0, 0);
|
||||
}
|
||||
|
||||
@CliAvailabilityIndicator({"desc"})
|
||||
|
||||
Reference in New Issue
Block a user