1
0

Implement Savepoints and required metadata timeline (#86)

- Introduce avro to save clean metadata with details about the last commit that was retained
- Save rollback metadata in the meta timeline
- Create savepoint metadata and add API to createSavepoint, deleteSavepoint and rollbackToSavepoint
- Savepointed commit should not be rolledback or cleaned or archived
- introduce cli commands to show, create and rollback to savepoints
- Write unit tests to test savepoints and rollbackToSavepoints
This commit is contained in:
prazanna
2017-03-13 15:12:03 -07:00
committed by GitHub
parent 69d3950a32
commit 6f36e1eaaf
27 changed files with 1423 additions and 130 deletions

View File

@@ -28,18 +28,19 @@ public class HoodiePrompt extends DefaultPromptProvider {
@Override
public String getPrompt() {
String tableName = HoodieCLI.tableMetadata.getTableConfig().getTableName();
switch (HoodieCLI.state) {
case INIT:
return "hoodie->";
case DATASET:
return "hoodie:" + tableName + "->";
case SYNC:
return "hoodie:" + tableName + " <==> "
+ HoodieCLI.syncTableMetadata.getTableConfig().getTableName() + "->";
}
if (HoodieCLI.tableMetadata != null)
if (HoodieCLI.tableMetadata != null) {
String tableName = HoodieCLI.tableMetadata.getTableConfig().getTableName();
switch (HoodieCLI.state) {
case INIT:
return "hoodie->";
case DATASET:
return "hoodie:" + tableName + "->";
case SYNC:
return "hoodie:" + tableName + " <==> "
+ HoodieCLI.syncTableMetadata.getTableConfig().getTableName() + "->";
}
return "hoodie:" + tableName + "->";
}
return "hoodie->";
}

View File

@@ -0,0 +1,112 @@
/*
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
*
* Licensed 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 com.uber.hoodie.cli.commands;
import com.uber.hoodie.avro.model.HoodieCleanMetadata;
import com.uber.hoodie.avro.model.HoodieCleanPartitionMetadata;
import com.uber.hoodie.cli.HoodieCLI;
import com.uber.hoodie.cli.HoodiePrintHelper;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
import com.uber.hoodie.common.table.timeline.HoodieInstant;
import com.uber.hoodie.common.util.AvroUtils;
import org.springframework.shell.core.CommandMarker;
import org.springframework.shell.core.annotation.CliAvailabilityIndicator;
import org.springframework.shell.core.annotation.CliCommand;
import org.springframework.shell.core.annotation.CliOption;
import org.springframework.stereotype.Component;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
@Component
public class CleansCommand implements CommandMarker {
@CliAvailabilityIndicator({"cleans show"})
public boolean isShowAvailable() {
return HoodieCLI.tableMetadata != null;
}
@CliAvailabilityIndicator({"cleans refresh"})
public boolean isRefreshAvailable() {
return HoodieCLI.tableMetadata != null;
}
@CliAvailabilityIndicator({"clean showpartitions"})
public boolean isCommitShowAvailable() {
return HoodieCLI.tableMetadata != null;
}
@CliCommand(value = "cleans show", help = "Show the cleans")
public String showCleans() throws IOException {
HoodieActiveTimeline activeTimeline = HoodieCLI.tableMetadata.getActiveTimeline();
HoodieTimeline timeline = activeTimeline.getCleanerTimeline().filterCompletedInstants();
List<HoodieInstant> cleans = timeline.getInstants().collect(Collectors.toList());
String[][] rows = new String[cleans.size()][];
Collections.reverse(cleans);
for (int i = 0; i < cleans.size(); i++) {
HoodieInstant clean = cleans.get(i);
HoodieCleanMetadata cleanMetadata =
AvroUtils.deserializeHoodieCleanMetadata(timeline.getInstantDetails(clean).get());
rows[i] = new String[] {clean.getTimestamp(), cleanMetadata.getEarliestCommitToRetain(),
String.valueOf(cleanMetadata.getTotalFilesDeleted()),
String.valueOf(cleanMetadata.getTimeTakenInMillis())};
}
return HoodiePrintHelper.print(
new String[] {"CleanTime", "EarliestCommandRetained", "Total Files Deleted",
"Total Time Taken"}, rows);
}
@CliCommand(value = "cleans refresh", help = "Refresh the commits")
public String refreshCleans() throws IOException {
HoodieTableMetaClient metadata =
new HoodieTableMetaClient(HoodieCLI.fs, HoodieCLI.tableMetadata.getBasePath());
HoodieCLI.setTableMetadata(metadata);
return "Metadata for table " + metadata.getTableConfig().getTableName() + " refreshed.";
}
@CliCommand(value = "clean showpartitions", help = "Show partition level details of a clean")
public String showCleanPartitions(
@CliOption(key = {"clean"}, help = "clean to show")
final String commitTime) throws Exception {
HoodieActiveTimeline activeTimeline = HoodieCLI.tableMetadata.getActiveTimeline();
HoodieTimeline timeline = activeTimeline.getCleanerTimeline().filterCompletedInstants();
HoodieInstant cleanInstant =
new HoodieInstant(false, HoodieTimeline.CLEAN_ACTION, commitTime);
if (!timeline.containsInstant(cleanInstant)) {
return "Clean " + commitTime + " not found in metadata " + timeline;
}
HoodieCleanMetadata cleanMetadata =
AvroUtils.deserializeHoodieCleanMetadata(timeline.getInstantDetails(cleanInstant).get());
List<String[]> rows = new ArrayList<>();
for (Map.Entry<String, HoodieCleanPartitionMetadata> entry : cleanMetadata.getPartitionMetadata().entrySet()) {
String path = entry.getKey();
HoodieCleanPartitionMetadata stats = entry.getValue();
String policy = stats.getPolicy();
String totalSuccessDeletedFiles = String.valueOf(stats.getSuccessDeleteFiles().size());
String totalFailedDeletedFiles = String.valueOf(stats.getFailedDeleteFiles().size());
rows.add(new String[] {path, policy, totalSuccessDeletedFiles, totalFailedDeletedFiles});
}
return HoodiePrintHelper.print(
new String[] {"Partition Path", "Cleaning policy", "Total Files Successfully Deleted",
"Total Failed Deletions"}, rows.toArray(new String[rows.size()][]));
}
}

View File

@@ -0,0 +1,160 @@
/*
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
*
* Licensed 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 com.uber.hoodie.cli.commands;
import com.uber.hoodie.HoodieWriteClient;
import com.uber.hoodie.avro.model.HoodieSavepointMetadata;
import com.uber.hoodie.cli.HoodieCLI;
import com.uber.hoodie.cli.HoodiePrintHelper;
import com.uber.hoodie.cli.utils.InputStreamConsumer;
import com.uber.hoodie.cli.utils.SparkUtil;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
import com.uber.hoodie.common.table.timeline.HoodieInstant;
import com.uber.hoodie.config.HoodieIndexConfig;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.index.HoodieIndex;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.launcher.SparkLauncher;
import org.springframework.shell.core.CommandMarker;
import org.springframework.shell.core.annotation.CliAvailabilityIndicator;
import org.springframework.shell.core.annotation.CliCommand;
import org.springframework.shell.core.annotation.CliOption;
import org.springframework.stereotype.Component;
import java.io.IOException;
import java.util.Collections;
import java.util.Date;
import java.util.List;
import java.util.stream.Collectors;
@Component
public class SavepointsCommand implements CommandMarker {
@CliAvailabilityIndicator({"savepoints show"})
public boolean isShowAvailable() {
return HoodieCLI.tableMetadata != null;
}
@CliAvailabilityIndicator({"savepoints refresh"})
public boolean isRefreshAvailable() {
return HoodieCLI.tableMetadata != null;
}
@CliAvailabilityIndicator({"savepoint create"})
public boolean isCreateSavepointAvailable() {
return HoodieCLI.tableMetadata != null;
}
@CliAvailabilityIndicator({"savepoint rollback"})
public boolean isRollbackToSavepointAvailable() {
return HoodieCLI.tableMetadata != null && !HoodieCLI.tableMetadata.getActiveTimeline().getSavePointTimeline().filterCompletedInstants().empty();
}
@CliCommand(value = "savepoints show", help = "Show the savepoints")
public String showSavepoints() throws IOException {
HoodieActiveTimeline activeTimeline = HoodieCLI.tableMetadata.getActiveTimeline();
HoodieTimeline timeline = activeTimeline.getSavePointTimeline().filterCompletedInstants();
List<HoodieInstant> commits = timeline.getInstants().collect(Collectors.toList());
String[][] rows = new String[commits.size()][];
Collections.reverse(commits);
for (int i = 0; i < commits.size(); i++) {
HoodieInstant commit = commits.get(i);
rows[i] = new String[] {commit.getTimestamp()};
}
return HoodiePrintHelper.print(new String[] {"SavepointTime"}, rows);
}
@CliCommand(value = "savepoint create", help = "Savepoint a commit")
public String savepoint(
@CliOption(key = {"commit"}, help = "Commit to savepoint")
final String commitTime,
@CliOption(key = {"user"}, help = "User who is creating the savepoint")
final String user,
@CliOption(key = {"comments"}, help = "Comments for creating the savepoint")
final String comments) throws Exception {
HoodieActiveTimeline activeTimeline = HoodieCLI.tableMetadata.getActiveTimeline();
HoodieTimeline timeline = activeTimeline.getCommitTimeline().filterCompletedInstants();
HoodieInstant
commitInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commitTime);
if (!timeline.containsInstant(commitInstant)) {
return "Commit " + commitTime + " not found in Commits " + timeline;
}
HoodieWriteClient client = createHoodieClient(null, HoodieCLI.tableMetadata.getBasePath());
HoodieSavepointMetadata metadata = new HoodieSavepointMetadata(user,
HoodieActiveTimeline.COMMIT_FORMATTER.format(new Date()), comments);
if (client.savepoint(commitTime, metadata)) {
// Refresh the current
refreshMetaClient();
return String.format("The commit \"%s\" has been savepointed.", commitTime);
}
return String.format("Failed: Could not savepoint commit \"%s\".", commitTime);
}
@CliCommand(value = "savepoint rollback", help = "Savepoint a commit")
public String rollbackToSavepoint(
@CliOption(key = {"savepoint"}, help = "Savepoint to rollback")
final String commitTime,
@CliOption(key = {"sparkProperties"}, help = "Spark Properites File Path")
final String sparkPropertiesPath) throws Exception {
HoodieActiveTimeline activeTimeline = HoodieCLI.tableMetadata.getActiveTimeline();
HoodieTimeline timeline = activeTimeline.getCommitTimeline().filterCompletedInstants();
HoodieInstant
commitInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commitTime);
if (!timeline.containsInstant(commitInstant)) {
return "Commit " + commitTime + " not found in Commits " + timeline;
}
SparkLauncher sparkLauncher = SparkUtil.initLauncher(sparkPropertiesPath);
sparkLauncher.addAppArgs(SparkMain.SparkCommand.ROLLBACK_TO_SAVEPOINT.toString(),
commitTime,
HoodieCLI.tableMetadata.getBasePath());
Process process = sparkLauncher.launch();
InputStreamConsumer.captureOutput(process);
int exitCode = process.waitFor();
// Refresh the current
refreshMetaClient();
if (exitCode != 0) {
return "Savepoint " + commitTime + " failed to roll back";
}
return "Savepoint " + commitTime + " rolled back";
}
@CliCommand(value = "savepoints refresh", help = "Refresh the savepoints")
public String refreshMetaClient() throws IOException {
HoodieTableMetaClient metadata =
new HoodieTableMetaClient(HoodieCLI.fs, HoodieCLI.tableMetadata.getBasePath());
HoodieCLI.setTableMetadata(metadata);
return "Metadata for table " + metadata.getTableConfig().getTableName() + " refreshed.";
}
private static HoodieWriteClient createHoodieClient(JavaSparkContext jsc, String basePath)
throws Exception {
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath)
.withIndexConfig(
HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build())
.build();
return new HoodieWriteClient(jsc, config, false);
}
}

View File

@@ -28,6 +28,8 @@ import org.apache.log4j.Logger;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.sql.SQLContext;
import java.util.Date;
public class SparkMain {
protected final static Logger LOG = Logger.getLogger(SparkMain.class);
@@ -38,7 +40,8 @@ public class SparkMain {
*/
enum SparkCommand {
ROLLBACK,
DEDUPLICATE
DEDUPLICATE,
ROLLBACK_TO_SAVEPOINT, SAVEPOINT
}
public static void main(String[] args) throws Exception {
@@ -55,6 +58,9 @@ public class SparkMain {
} else if(SparkCommand.DEDUPLICATE.equals(cmd)) {
assert (args.length == 4);
returnCode = deduplicatePartitionPath(jsc, args[1], args[2], args[3]);
} else if(SparkCommand.ROLLBACK_TO_SAVEPOINT.equals(cmd)) {
assert (args.length == 3);
returnCode = rollbackToSavepoint(jsc, args[1], args[2]);
}
System.exit(returnCode);
@@ -76,11 +82,23 @@ public class SparkMain {
HoodieWriteClient client = createHoodieClient(jsc, basePath);
if (client.rollback(commitTime)) {
LOG.info(String.format("The commit \"%s\" rolled back.", commitTime));
return -1;
return 0;
} else {
LOG.info(String.format("The commit \"%s\" failed to roll back.", commitTime));
return -1;
}
}
private static int rollbackToSavepoint(JavaSparkContext jsc, String savepointTime, String basePath)
throws Exception {
HoodieWriteClient client = createHoodieClient(jsc, basePath);
if (client.rollbackToSavepoint(savepointTime)) {
LOG.info(String.format("The commit \"%s\" rolled back.", savepointTime));
return 0;
} else {
LOG.info(String.format("The commit \"%s\" failed to roll back.", savepointTime));
return -1;
}
return 0;
}
private static HoodieWriteClient createHoodieClient(JavaSparkContext jsc, String basePath)