rows) {
+ return print(rowHeader, fieldNameToConverterMap, withRowNo, sortByField, isDescending, limit, headerOnly, rows, "");
}
/**
* Serialize Table to printable string and also export a temporary view to easily write sql queries.
+ *
+ * Ideally, exporting view needs to be outside PrintHelper, but all commands use this. So this is easy
+ * way to add support for all commands
*
+ * @param rowHeader Row Header
+ * @param fieldNameToConverterMap Field Specific Converters
+ * @param sortByField Sorting field
+ * @param isDescending Order
+ * @param limit Limit
+ * @param headerOnly Headers only
+ * @param rows List of rows
+ * @param tempTableName table name to export
+ * @return Serialized form for printing
+ */
+ public static String print(
+ TableHeader rowHeader, Map> fieldNameToConverterMap,
+ String sortByField, boolean isDescending, Integer limit, boolean headerOnly,
+ List rows, String tempTableName) {
+ return print(rowHeader, fieldNameToConverterMap, false, sortByField, isDescending, limit,
+ headerOnly, rows, tempTableName);
+ }
+
+ /**
+ * Serialize Table to printable string and also export a temporary view to easily write sql queries.
+ *
* Ideally, exporting view needs to be outside PrintHelper, but all commands use this. So this is easy
* way to add support for all commands
*
- * @param rowHeader Row Header
+ * @param rowHeader Row Header
* @param fieldNameToConverterMap Field Specific Converters
- * @param sortByField Sorting field
- * @param isDescending Order
- * @param limit Limit
- * @param headerOnly Headers only
- * @param rows List of rows
- * @param tempTableName table name to export
+ * @param withRowNo Whether to add row number
+ * @param sortByField Sorting field
+ * @param isDescending Order
+ * @param limit Limit
+ * @param headerOnly Headers only
+ * @param rows List of rows
+ * @param tempTableName table name to export
* @return Serialized form for printing
*/
- public static String print(TableHeader rowHeader, Map> fieldNameToConverterMap,
- String sortByField, boolean isDescending, Integer limit, boolean headerOnly, List rows,
- String tempTableName) {
+ public static String print(
+ TableHeader rowHeader, Map> fieldNameToConverterMap,
+ boolean withRowNo, String sortByField, boolean isDescending, Integer limit, boolean headerOnly,
+ List rows, String tempTableName) {
if (headerOnly) {
return HoodiePrintHelper.print(rowHeader);
@@ -97,7 +143,8 @@ public static String print(TableHeader rowHeader, Map";
+ return new AttributedString("hudi->");
case TABLE:
- return "hudi:" + tableName + "->";
+ return new AttributedString("hudi:" + tableName + "->");
case SYNC:
- return "hudi:" + tableName + " <==> " + HoodieCLI.syncTableMetadata.getTableConfig().getTableName() + "->";
+ return new AttributedString("hudi:" + tableName + " <==> " + HoodieCLI.syncTableMetadata.getTableConfig().getTableName() + "->");
default:
- return "hudi:" + tableName + "->";
+ return new AttributedString("hudi:" + tableName + "->");
}
}
- return "hudi->";
+ return new AttributedString("hudi->");
}
-
- @Override
- public String getProviderName() {
- return "Hoodie provider";
- }
-
}
diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/HoodieSplashScreen.java b/hudi-cli/src/main/java/org/apache/hudi/cli/HoodieSplashScreen.java
deleted file mode 100644
index f2a458c196c94..0000000000000
--- a/hudi-cli/src/main/java/org/apache/hudi/cli/HoodieSplashScreen.java
+++ /dev/null
@@ -1,72 +0,0 @@
-/*
- * 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.cli;
-
-import org.springframework.core.Ordered;
-import org.springframework.core.annotation.Order;
-import org.springframework.shell.plugin.support.DefaultBannerProvider;
-import org.springframework.shell.support.util.OsUtils;
-import org.springframework.stereotype.Component;
-
-/**
- * This class is responsible to print the splash screen at the start of the application.
- */
-@Component
-@Order(Ordered.HIGHEST_PRECEDENCE)
-public class HoodieSplashScreen extends DefaultBannerProvider {
-
- static {
- System.out.println("HoodieSplashScreen loaded");
- }
-
- private static String screen = "===================================================================" + OsUtils.LINE_SEPARATOR
- + "* ___ ___ *" + OsUtils.LINE_SEPARATOR
- + "* /\\__\\ ___ /\\ \\ ___ *" + OsUtils.LINE_SEPARATOR
- + "* / / / /\\__\\ / \\ \\ /\\ \\ *" + OsUtils.LINE_SEPARATOR
- + "* / /__/ / / / / /\\ \\ \\ \\ \\ \\ *" + OsUtils.LINE_SEPARATOR
- + "* / \\ \\ ___ / / / / / \\ \\__\\ / \\__\\ *" + OsUtils.LINE_SEPARATOR
- + "* / /\\ \\ /\\__\\ / /__/ ___ / /__/ \\ |__| / /\\/__/ *" + OsUtils.LINE_SEPARATOR
- + "* \\/ \\ \\/ / / \\ \\ \\ /\\__\\ \\ \\ \\ / / / /\\/ / / *" + OsUtils.LINE_SEPARATOR
- + "* \\ / / \\ \\ / / / \\ \\ / / / \\ /__/ *" + OsUtils.LINE_SEPARATOR
- + "* / / / \\ \\/ / / \\ \\/ / / \\ \\__\\ *" + OsUtils.LINE_SEPARATOR
- + "* / / / \\ / / \\ / / \\/__/ *" + OsUtils.LINE_SEPARATOR
- + "* \\/__/ \\/__/ \\/__/ Apache Hudi CLI *" + OsUtils.LINE_SEPARATOR
- + "* *" + OsUtils.LINE_SEPARATOR
- + "===================================================================" + OsUtils.LINE_SEPARATOR;
-
- @Override
- public String getBanner() {
- return screen;
- }
-
- @Override
- public String getVersion() {
- return "1.0";
- }
-
- @Override
- public String getWelcomeMessage() {
- return "Welcome to Apache Hudi CLI. Please type help if you are looking for help. ";
- }
-
- @Override
- public String getProviderName() {
- return "Hoodie Banner";
- }
-}
diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/HoodieTableHeaderFields.java b/hudi-cli/src/main/java/org/apache/hudi/cli/HoodieTableHeaderFields.java
index f32b7bc36c9bd..e6016e4cc1cb7 100644
--- a/hudi-cli/src/main/java/org/apache/hudi/cli/HoodieTableHeaderFields.java
+++ b/hudi-cli/src/main/java/org/apache/hudi/cli/HoodieTableHeaderFields.java
@@ -22,6 +22,7 @@
* Fields of print table header.
*/
public class HoodieTableHeaderFields {
+ public static final String HEADER_ROW_NO = "No.";
public static final String HEADER_PARTITION = "Partition";
public static final String HEADER_INSTANT = "Instant";
public static final String HEADER_PARTITION_PATH = HEADER_PARTITION + " Path";
@@ -83,6 +84,8 @@ public class HoodieTableHeaderFields {
public static final String HEADER_HOODIE_PROPERTY = "Property";
public static final String HEADER_OLD_VALUE = "Old Value";
public static final String HEADER_NEW_VALUE = "New Value";
+ public static final String HEADER_TEXT_METAFILE_PRESENT = "Text Metafile present ?";
+ public static final String HEADER_BASE_METAFILE_PRESENT = "Base Metafile present ?";
/**
* Fields of Savepoints.
@@ -126,6 +129,9 @@ public class HoodieTableHeaderFields {
public static final String HEADER_TOTAL_RECORDS_INSERTED = "Total Records Inserted";
public static final String HEADER_TOTAL_RECORDS_UPDATED = "Total Records Updated";
public static final String HEADER_TOTAL_ERRORS = "Total Errors";
+ public static final String HEADER_TOTAL_RECORDS_WRITTEN_COMMIT = "Total Records Written for entire commit";
+ public static final String HEADER_TOTAL_BYTES_WRITTEN_COMMIT = "Total Bytes Written for entire commit";
+ public static final String HEADER_AVG_REC_SIZE_COMMIT = "Avg record size for entire commit";
/**
* Fields of commit metadata.
@@ -140,4 +146,71 @@ public class HoodieTableHeaderFields {
public static final String HEADER_TOTAL_ROLLBACK_BLOCKS = "Total Rollback Blocks";
public static final String HEADER_TOTAL_LOG_RECORDS = "Total Log Records";
public static final String HEADER_TOTAL_UPDATED_RECORDS_COMPACTED = "Total Updated Records Compacted";
+
+ /**
+ * Fields of Compaction.
+ */
+ public static final String HEADER_INSTANT_BLANK_TIME = "Instant Time";
+ public static final String HEADER_FILE_PATH = "File Path";
+ public static final String HEADER_COMPACTION_INSTANT_TIME = "Compaction " + HEADER_INSTANT_BLANK_TIME;
+ public static final String HEADER_STATE = "State";
+ public static final String HEADER_TOTAL_FILES_TO_BE_COMPACTED = "Total FileIds to be Compacted";
+ public static final String HEADER_EXTRA_METADATA = "Extra Metadata";
+ public static final String HEADER_DATA_FILE_PATH = "Data " + HEADER_FILE_PATH;
+ public static final String HEADER_TOTAL_DELTA_FILES = "Total " + HEADER_DELTA_FILES;
+ public static final String HEADER_METRICS = "getMetrics";
+ public static final String HEADER_BASE_INSTANT_TIME = "Base " + HEADER_INSTANT_BLANK_TIME;
+ public static final String HEADER_BASE_DATA_FILE = "Base Data File";
+ public static final String HEADER_VALID = "Valid";
+ public static final String HEADER_ERROR = "Error";
+ public static final String HEADER_SOURCE_FILE_PATH = "Source " + HEADER_FILE_PATH;
+ public static final String HEADER_DESTINATION_FILE_PATH = "Destination " + HEADER_FILE_PATH;
+ public static final String HEADER_RENAME_EXECUTED = "Rename Executed?";
+ public static final String HEADER_RENAME_SUCCEEDED = "Rename Succeeded?";
+
+ /**
+ * Fields of timeline command output
+ */
+ public static final String HEADER_REQUESTED_TIME = "Requested\nTime";
+ public static final String HEADER_INFLIGHT_TIME = "Inflight\nTime";
+ public static final String HEADER_COMPLETED_TIME = "Completed\nTime";
+ public static final String HEADER_ROLLBACK_INFO = "Rollback Info";
+ public static final String HEADER_MT_PREFIX = "MT\n";
+ public static final String HEADER_MT_ACTION = HEADER_MT_PREFIX + HEADER_ACTION;
+ public static final String HEADER_MT_STATE = HEADER_MT_PREFIX + HEADER_STATE;
+ public static final String HEADER_MT_REQUESTED_TIME = HEADER_MT_PREFIX + HEADER_REQUESTED_TIME;
+ public static final String HEADER_MT_INFLIGHT_TIME = HEADER_MT_PREFIX + HEADER_INFLIGHT_TIME;
+ public static final String HEADER_MT_COMPLETED_TIME = HEADER_MT_PREFIX + HEADER_COMPLETED_TIME;
+
+ public static TableHeader getTableHeader() {
+ return new TableHeader()
+ .addTableHeaderField(HoodieTableHeaderFields.HEADER_COMMIT_TIME)
+ .addTableHeaderField(HoodieTableHeaderFields.HEADER_TOTAL_BYTES_WRITTEN)
+ .addTableHeaderField(HoodieTableHeaderFields.HEADER_TOTAL_FILES_ADDED)
+ .addTableHeaderField(HoodieTableHeaderFields.HEADER_TOTAL_FILES_UPDATED)
+ .addTableHeaderField(HoodieTableHeaderFields.HEADER_TOTAL_PARTITIONS_WRITTEN)
+ .addTableHeaderField(HoodieTableHeaderFields.HEADER_TOTAL_RECORDS_WRITTEN)
+ .addTableHeaderField(HoodieTableHeaderFields.HEADER_TOTAL_UPDATE_RECORDS_WRITTEN)
+ .addTableHeaderField(HoodieTableHeaderFields.HEADER_TOTAL_ERRORS);
+ }
+
+ public static TableHeader getTableHeaderWithExtraMetadata() {
+ return new TableHeader()
+ .addTableHeaderField(HoodieTableHeaderFields.HEADER_ACTION)
+ .addTableHeaderField(HoodieTableHeaderFields.HEADER_INSTANT)
+ .addTableHeaderField(HoodieTableHeaderFields.HEADER_PARTITION)
+ .addTableHeaderField(HoodieTableHeaderFields.HEADER_FILE_ID)
+ .addTableHeaderField(HoodieTableHeaderFields.HEADER_PREVIOUS_COMMIT)
+ .addTableHeaderField(HoodieTableHeaderFields.HEADER_NUM_WRITES)
+ .addTableHeaderField(HoodieTableHeaderFields.HEADER_NUM_INSERTS)
+ .addTableHeaderField(HoodieTableHeaderFields.HEADER_NUM_DELETES)
+ .addTableHeaderField(HoodieTableHeaderFields.HEADER_NUM_UPDATE_WRITES)
+ .addTableHeaderField(HoodieTableHeaderFields.HEADER_TOTAL_ERRORS)
+ .addTableHeaderField(HoodieTableHeaderFields.HEADER_TOTAL_LOG_BLOCKS)
+ .addTableHeaderField(HoodieTableHeaderFields.HEADER_TOTAL_CORRUPT_LOG_BLOCKS)
+ .addTableHeaderField(HoodieTableHeaderFields.HEADER_TOTAL_ROLLBACK_BLOCKS)
+ .addTableHeaderField(HoodieTableHeaderFields.HEADER_TOTAL_LOG_RECORDS)
+ .addTableHeaderField(HoodieTableHeaderFields.HEADER_TOTAL_UPDATED_RECORDS_COMPACTED)
+ .addTableHeaderField(HoodieTableHeaderFields.HEADER_TOTAL_BYTES_WRITTEN);
+ }
}
diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/Main.java b/hudi-cli/src/main/java/org/apache/hudi/cli/Main.java
index e924be9e50f52..e98707800196e 100644
--- a/hudi-cli/src/main/java/org/apache/hudi/cli/Main.java
+++ b/hudi-cli/src/main/java/org/apache/hudi/cli/Main.java
@@ -18,18 +18,19 @@
package org.apache.hudi.cli;
-import org.springframework.shell.Bootstrap;
+import org.springframework.boot.SpringApplication;
+import org.springframework.boot.autoconfigure.SpringBootApplication;
import java.io.IOException;
/**
* Main class that delegates to Spring Shell's Bootstrap class in order to simplify debugging inside an IDE.
*/
+@SpringBootApplication
public class Main {
public static void main(String[] args) throws IOException {
System.out.println("Main called");
- new HoodieSplashScreen();
- Bootstrap.main(args);
+ SpringApplication.run(Main.class, args);
}
}
diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/Table.java b/hudi-cli/src/main/java/org/apache/hudi/cli/Table.java
index 8158eef8d5f84..70e8a9740301c 100644
--- a/hudi-cli/src/main/java/org/apache/hudi/cli/Table.java
+++ b/hudi-cli/src/main/java/org/apache/hudi/cli/Table.java
@@ -37,6 +37,8 @@ public class Table implements Iterable> {
// Header for this table
private final TableHeader rowHeader;
+ // Whether to print row number
+ private final boolean addRowNo;
// User-specified conversions before rendering
private final Map> fieldNameToConverterMap;
// Option attribute to track sorting field
@@ -49,12 +51,17 @@ public class Table implements Iterable> {
private final List> rawRows;
// Flag to determine if all the rows have been added
private boolean finishedAdding = false;
- // Rows ready for Rendering
+ // Headers ready for rendering
+ private TableHeader renderHeaders;
+ // Rows ready for rendering
private List> renderRows;
- public Table(TableHeader rowHeader, Map> fieldNameToConverterMap,
- Option orderingFieldNameOptional, Option isDescendingOptional, Option limitOptional) {
+ public Table(
+ TableHeader rowHeader, Map> fieldNameToConverterMap,
+ boolean addRowNo, Option orderingFieldNameOptional,
+ Option isDescendingOptional, Option limitOptional) {
this.rowHeader = rowHeader;
+ this.addRowNo = addRowNo;
this.fieldNameToConverterMap = fieldNameToConverterMap;
this.orderingFieldNameOptional = orderingFieldNameOptional;
this.isDescendingOptional = isDescendingOptional;
@@ -64,7 +71,7 @@ public Table(TableHeader rowHeader, Map> fieldN
/**
* Main API to add row to the table.
- *
+ *
* @param row Row
*/
public Table add(List row) {
@@ -134,15 +141,34 @@ private List> orderRows() {
private void sortAndLimit() {
this.renderRows = new ArrayList<>();
final int limit = this.limitOptional.orElse(rawRows.size());
- final List> orderedRows = orderRows();
- renderRows = orderedRows.stream().limit(limit).map(row -> IntStream.range(0, rowHeader.getNumFields()).mapToObj(idx -> {
- String fieldName = rowHeader.get(idx);
- if (fieldNameToConverterMap.containsKey(fieldName)) {
- return fieldNameToConverterMap.get(fieldName).apply(row.get(idx));
+ // Row number is added here if enabled
+ final List> rawOrderedRows = orderRows();
+ final List> orderedRows;
+ if (addRowNo) {
+ orderedRows = new ArrayList<>();
+ int rowNo = 0;
+ for (List row : rawOrderedRows) {
+ List newRow = new ArrayList<>();
+ newRow.add(rowNo++);
+ newRow.addAll(row);
+ orderedRows.add(newRow);
}
- Object v = row.get(idx);
- return v == null ? "null" : v.toString();
- }).collect(Collectors.toList())).collect(Collectors.toList());
+ } else {
+ orderedRows = rawOrderedRows;
+ }
+ renderHeaders = addRowNo
+ ? new TableHeader().addTableHeaderField(HoodieTableHeaderFields.HEADER_ROW_NO)
+ .addTableHeaderFields(rowHeader)
+ : rowHeader;
+ renderRows = orderedRows.stream().limit(limit)
+ .map(row -> IntStream.range(0, renderHeaders.getNumFields()).mapToObj(idx -> {
+ String fieldName = renderHeaders.get(idx);
+ if (fieldNameToConverterMap.containsKey(fieldName)) {
+ return fieldNameToConverterMap.get(fieldName).apply(row.get(idx));
+ }
+ Object v = row.get(idx);
+ return v == null ? "null" : v.toString();
+ }).collect(Collectors.toList())).collect(Collectors.toList());
}
@Override
@@ -162,6 +188,9 @@ public void forEach(Consumer super List> action) {
}
public List getFieldNames() {
+ if (renderHeaders != null) {
+ return renderHeaders.getFieldNames();
+ }
return rowHeader.getFieldNames();
}
diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/TableHeader.java b/hudi-cli/src/main/java/org/apache/hudi/cli/TableHeader.java
index 8ec392d1abfe3..ee17480a30da2 100644
--- a/hudi-cli/src/main/java/org/apache/hudi/cli/TableHeader.java
+++ b/hudi-cli/src/main/java/org/apache/hudi/cli/TableHeader.java
@@ -39,6 +39,16 @@ public TableHeader addTableHeaderField(String fieldName) {
return this;
}
+ /**
+ * Add fields from another {@link TableHeader} instance.
+ *
+ * @param tableHeader {@link TableHeader} instance.
+ */
+ public TableHeader addTableHeaderFields(TableHeader tableHeader) {
+ fieldNames.addAll(tableHeader.getFieldNames());
+ return this;
+ }
+
/**
* Get all field names.
*/
diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/ArchivedCommitsCommand.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/ArchivedCommitsCommand.java
index 102fcc2ae7a63..dcd6a2cf3c8e9 100644
--- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/ArchivedCommitsCommand.java
+++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/ArchivedCommitsCommand.java
@@ -18,6 +18,11 @@
package org.apache.hudi.cli.commands;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.avro.generic.IndexedRecord;
+import org.apache.avro.specific.SpecificData;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
import org.apache.hudi.avro.model.HoodieArchivedMetaEntry;
import org.apache.hudi.avro.model.HoodieCommitMetadata;
import org.apache.hudi.cli.HoodieCLI;
@@ -30,17 +35,11 @@
import org.apache.hudi.common.table.log.HoodieLogFormat.Reader;
import org.apache.hudi.common.table.log.block.HoodieAvroDataBlock;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
+import org.apache.hudi.common.util.ClosableIterator;
import org.apache.hudi.common.util.Option;
-
-import org.apache.avro.generic.GenericRecord;
-import org.apache.avro.generic.IndexedRecord;
-import org.apache.avro.specific.SpecificData;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.Path;
-import org.springframework.shell.core.CommandMarker;
-import org.springframework.shell.core.annotation.CliCommand;
-import org.springframework.shell.core.annotation.CliOption;
-import org.springframework.stereotype.Component;
+import org.springframework.shell.standard.ShellComponent;
+import org.springframework.shell.standard.ShellMethod;
+import org.springframework.shell.standard.ShellOption;
import java.io.IOException;
import java.util.ArrayList;
@@ -51,17 +50,17 @@
/**
* CLI command to display archived commits and stats if available.
*/
-@Component
-public class ArchivedCommitsCommand implements CommandMarker {
+@ShellComponent
+public class ArchivedCommitsCommand {
- @CliCommand(value = "show archived commit stats", help = "Read commits from archived files and show details")
+ @ShellMethod(key = "show archived commit stats", value = "Read commits from archived files and show details")
public String showArchivedCommits(
- @CliOption(key = {"archiveFolderPattern"}, help = "Archive Folder", unspecifiedDefaultValue = "") String folder,
- @CliOption(key = {"limit"}, help = "Limit commits", unspecifiedDefaultValue = "-1") final Integer limit,
- @CliOption(key = {"sortBy"}, help = "Sorting Field", unspecifiedDefaultValue = "") final String sortByField,
- @CliOption(key = {"desc"}, help = "Ordering", unspecifiedDefaultValue = "false") final boolean descending,
- @CliOption(key = {"headeronly"}, help = "Print Header Only",
- unspecifiedDefaultValue = "false") final boolean headerOnly)
+ @ShellOption(value = {"--archiveFolderPattern"}, help = "Archive Folder", defaultValue = "") String folder,
+ @ShellOption(value = {"--limit"}, help = "Limit commits", defaultValue = "-1") final Integer limit,
+ @ShellOption(value = {"--sortBy"}, help = "Sorting Field", defaultValue = "") final String sortByField,
+ @ShellOption(value = {"--desc"}, help = "Ordering", defaultValue = "false") final boolean descending,
+ @ShellOption(value = {"--headeronly"}, help = "Print Header Only",
+ defaultValue = "false") final boolean headerOnly)
throws IOException {
System.out.println("===============> Showing only " + limit + " archived commits <===============");
String basePath = HoodieCLI.getTableMetaClient().getBasePath();
@@ -80,8 +79,7 @@ public String showArchivedCommits(
// read the avro blocks
while (reader.hasNext()) {
HoodieAvroDataBlock blk = (HoodieAvroDataBlock) reader.next();
- List records = blk.getRecords();
- readRecords.addAll(records);
+ blk.getRecordIterator().forEachRemaining(readRecords::add);
}
List readCommits = readRecords.stream().map(r -> (GenericRecord) r)
.filter(r -> r.get("actionType").toString().equals(HoodieTimeline.COMMIT_ACTION)
@@ -128,15 +126,15 @@ public String showArchivedCommits(
return HoodiePrintHelper.print(header, new HashMap<>(), sortByField, descending, limit, headerOnly, allStats);
}
- @CliCommand(value = "show archived commits", help = "Read commits from archived files and show details")
+ @ShellMethod(key = "show archived commits", value = "Read commits from archived files and show details")
public String showCommits(
- @CliOption(key = {"skipMetadata"}, help = "Skip displaying commit metadata",
- unspecifiedDefaultValue = "true") boolean skipMetadata,
- @CliOption(key = {"limit"}, help = "Limit commits", unspecifiedDefaultValue = "10") final Integer limit,
- @CliOption(key = {"sortBy"}, help = "Sorting Field", unspecifiedDefaultValue = "") final String sortByField,
- @CliOption(key = {"desc"}, help = "Ordering", unspecifiedDefaultValue = "false") final boolean descending,
- @CliOption(key = {"headeronly"}, help = "Print Header Only",
- unspecifiedDefaultValue = "false") final boolean headerOnly)
+ @ShellOption(value = {"--skipMetadata"}, help = "Skip displaying commit metadata",
+ defaultValue = "true") boolean skipMetadata,
+ @ShellOption(value = {"--limit"}, help = "Limit commits", defaultValue = "10") final Integer limit,
+ @ShellOption(value = {"--sortBy"}, help = "Sorting Field", defaultValue = "") final String sortByField,
+ @ShellOption(value = {"--desc"}, help = "Ordering", defaultValue = "false") final boolean descending,
+ @ShellOption(value = {"--headeronly"}, help = "Print Header Only",
+ defaultValue = "false") final boolean headerOnly)
throws IOException {
System.out.println("===============> Showing only " + limit + " archived commits <===============");
@@ -155,8 +153,9 @@ public String showCommits(
// read the avro blocks
while (reader.hasNext()) {
HoodieAvroDataBlock blk = (HoodieAvroDataBlock) reader.next();
- List records = blk.getRecords();
- readRecords.addAll(records);
+ try (ClosableIterator recordItr = blk.getRecordIterator()) {
+ recordItr.forEachRemaining(readRecords::add);
+ }
}
List readCommits = readRecords.stream().map(r -> (GenericRecord) r)
.map(r -> readCommit(r, skipMetadata)).collect(Collectors.toList());
@@ -199,12 +198,12 @@ private Comparable[] readCommit(GenericRecord record, boolean skipMetadata) {
case HoodieTimeline.COMPACTION_ACTION:
return commitDetail(record, "hoodieCompactionMetadata", skipMetadata);
default: {
- return new Comparable[]{};
+ return new Comparable[] {};
}
}
} catch (Exception e) {
e.printStackTrace();
- return new Comparable[]{};
+ return new Comparable[] {};
}
}
}
diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/BootstrapCommand.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/BootstrapCommand.java
index 015743d2f299f..98cf9fc0d9067 100644
--- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/BootstrapCommand.java
+++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/BootstrapCommand.java
@@ -30,60 +30,56 @@
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.utilities.UtilHelpers;
-
import org.apache.spark.launcher.SparkLauncher;
import org.apache.spark.util.Utils;
-import org.springframework.shell.core.CommandMarker;
-import org.springframework.shell.core.annotation.CliCommand;
-import org.springframework.shell.core.annotation.CliOption;
-import org.springframework.stereotype.Component;
+import org.springframework.shell.standard.ShellComponent;
+import org.springframework.shell.standard.ShellMethod;
+import org.springframework.shell.standard.ShellOption;
+import scala.collection.JavaConverters;
import java.io.IOException;
import java.net.URISyntaxException;
-import java.util.Arrays;
import java.util.ArrayList;
+import java.util.Arrays;
import java.util.HashMap;
import java.util.List;
import java.util.stream.Collectors;
-import scala.collection.JavaConverters;
-
/**
* CLI command to perform bootstrap action & display bootstrap index.
*/
-@Component
-public class BootstrapCommand implements CommandMarker {
+@ShellComponent
+public class BootstrapCommand {
- @CliCommand(value = "bootstrap run", help = "Run a bootstrap action for current Hudi table")
+ @ShellMethod(key = "bootstrap run", value = "Run a bootstrap action for current Hudi table")
public String bootstrap(
- @CliOption(key = {"srcPath"}, mandatory = true, help = "Bootstrap source data path of the table") final String srcPath,
- @CliOption(key = {"targetPath"}, mandatory = true,
- help = "Base path for the target hoodie table") final String targetPath,
- @CliOption(key = {"tableName"}, mandatory = true, help = "Hoodie table name") final String tableName,
- @CliOption(key = {"tableType"}, mandatory = true, help = "Hoodie table type") final String tableType,
- @CliOption(key = {"rowKeyField"}, mandatory = true, help = "Record key columns for bootstrap data") final String rowKeyField,
- @CliOption(key = {"partitionPathField"}, unspecifiedDefaultValue = "",
+ @ShellOption(value = {"--srcPath"}, help = "Bootstrap source data path of the table") final String srcPath,
+ @ShellOption(value = {"--targetPath"}, help = "Base path for the target hoodie table") final String targetPath,
+ @ShellOption(value = {"--tableName"}, help = "Hoodie table name") final String tableName,
+ @ShellOption(value = {"--tableType"}, help = "Hoodie table type") final String tableType,
+ @ShellOption(value = {"--rowKeyField"}, help = "Record key columns for bootstrap data") final String rowKeyField,
+ @ShellOption(value = {"--partitionPathField"}, defaultValue = "",
help = "Partition fields for bootstrap source data") final String partitionPathField,
- @CliOption(key = {"bootstrapIndexClass"}, unspecifiedDefaultValue = "org.apache.hudi.common.bootstrap.index.HFileBootstrapIndex",
+ @ShellOption(value = {"--bootstrapIndexClass"}, defaultValue = "org.apache.hudi.common.bootstrap.index.HFileBootstrapIndex",
help = "Bootstrap Index Class") final String bootstrapIndexClass,
- @CliOption(key = {"selectorClass"}, unspecifiedDefaultValue = "org.apache.hudi.client.bootstrap.selector.MetadataOnlyBootstrapModeSelector",
+ @ShellOption(value = {"--selectorClass"}, defaultValue = "org.apache.hudi.client.bootstrap.selector.MetadataOnlyBootstrapModeSelector",
help = "Selector class for bootstrap") final String selectorClass,
- @CliOption(key = {"keyGeneratorClass"}, unspecifiedDefaultValue = "org.apache.hudi.keygen.SimpleKeyGenerator",
+ @ShellOption(value = {"--keyGeneratorClass"}, defaultValue = "org.apache.hudi.keygen.SimpleKeyGenerator",
help = "Key generator class for bootstrap") final String keyGeneratorClass,
- @CliOption(key = {"fullBootstrapInputProvider"}, unspecifiedDefaultValue = "org.apache.hudi.bootstrap.SparkParquetBootstrapDataProvider",
+ @ShellOption(value = {"--fullBootstrapInputProvider"}, defaultValue = "org.apache.hudi.bootstrap.SparkParquetBootstrapDataProvider",
help = "Class for Full bootstrap input provider") final String fullBootstrapInputProvider,
- @CliOption(key = {"schemaProviderClass"}, unspecifiedDefaultValue = "",
+ @ShellOption(value = {"--schemaProviderClass"}, defaultValue = "",
help = "SchemaProvider to attach schemas to bootstrap source data") final String schemaProviderClass,
- @CliOption(key = {"payloadClass"}, unspecifiedDefaultValue = "org.apache.hudi.common.model.OverwriteWithLatestAvroPayload",
+ @ShellOption(value = {"--payloadClass"}, defaultValue = "org.apache.hudi.common.model.OverwriteWithLatestAvroPayload",
help = "Payload Class") final String payloadClass,
- @CliOption(key = {"parallelism"}, unspecifiedDefaultValue = "1500", help = "Bootstrap writer parallelism") final int parallelism,
- @CliOption(key = {"sparkMaster"}, unspecifiedDefaultValue = "", help = "Spark Master") String master,
- @CliOption(key = {"sparkMemory"}, unspecifiedDefaultValue = "4G", help = "Spark executor memory") final String sparkMemory,
- @CliOption(key = {"enableHiveSync"}, unspecifiedDefaultValue = "false", help = "Enable Hive sync") final Boolean enableHiveSync,
- @CliOption(key = {"propsFilePath"}, help = "path to properties file on localfs or dfs with configurations for hoodie client for importing",
- unspecifiedDefaultValue = "") final String propsFilePath,
- @CliOption(key = {"hoodieConfigs"}, help = "Any configuration that can be set in the properties file can be passed here in the form of an array",
- unspecifiedDefaultValue = "") final String[] configs)
+ @ShellOption(value = {"--parallelism"}, defaultValue = "1500", help = "Bootstrap writer parallelism") final int parallelism,
+ @ShellOption(value = {"--sparkMaster"}, defaultValue = "", help = "Spark Master") String master,
+ @ShellOption(value = {"--sparkMemory"}, defaultValue = "4G", help = "Spark executor memory") final String sparkMemory,
+ @ShellOption(value = {"--enableHiveSync"}, defaultValue = "false", help = "Enable Hive sync") final Boolean enableHiveSync,
+ @ShellOption(value = {"--propsFilePath"}, help = "path to properties file on localfs or dfs with configurations for hoodie client for importing",
+ defaultValue = "") final String propsFilePath,
+ @ShellOption(value = {"--hoodieConfigs"}, help = "Any configuration that can be set in the properties file can be passed here in the form of an array",
+ defaultValue = "") final String[] configs)
throws IOException, InterruptedException, URISyntaxException {
String sparkPropertiesPath =
@@ -106,15 +102,14 @@ public String bootstrap(
return "Bootstrapped source data as Hudi dataset";
}
- @CliCommand(value = "bootstrap index showmapping", help = "Show bootstrap index mapping")
+ @ShellMethod(key = "bootstrap index showmapping", value = "Show bootstrap index mapping")
public String showBootstrapIndexMapping(
- @CliOption(key = {"partitionPath"}, unspecifiedDefaultValue = "", help = "A valid partition path") String partitionPath,
- @CliOption(key = {"fileIds"}, unspecifiedDefaultValue = "", help = "Valid fileIds split by comma") String fileIds,
- @CliOption(key = {"limit"}, unspecifiedDefaultValue = "-1", help = "Limit rows to be displayed") Integer limit,
- @CliOption(key = {"sortBy"}, unspecifiedDefaultValue = "", help = "Sorting Field") final String sortByField,
- @CliOption(key = {"desc"}, unspecifiedDefaultValue = "false", help = "Ordering") final boolean descending,
- @CliOption(key = {"headeronly"}, unspecifiedDefaultValue = "false", help = "Print Header Only")
- final boolean headerOnly) {
+ @ShellOption(value = {"--partitionPath"}, defaultValue = "", help = "A valid partition path") String partitionPath,
+ @ShellOption(value = {"--fileIds"}, defaultValue = "", help = "Valid fileIds split by comma") String fileIds,
+ @ShellOption(value = {"--limit"}, defaultValue = "-1", help = "Limit rows to be displayed") Integer limit,
+ @ShellOption(value = {"--sortBy"}, defaultValue = "", help = "Sorting Field") final String sortByField,
+ @ShellOption(value = {"--desc"}, defaultValue = "false", help = "Ordering") final boolean descending,
+ @ShellOption(value = {"--headeronly"}, defaultValue = "false", help = "Print Header Only") final boolean headerOnly) {
if (partitionPath.isEmpty() && !fileIds.isEmpty()) {
throw new IllegalStateException("PartitionPath is mandatory when passing fileIds.");
@@ -152,7 +147,7 @@ public String showBootstrapIndexMapping(
limit, headerOnly, rows);
}
- @CliCommand(value = "bootstrap index showpartitions", help = "Show bootstrap indexed partitions")
+ @ShellMethod(key = "bootstrap index showpartitions", value = "Show bootstrap indexed partitions")
public String showBootstrapIndexPartitions() {
BootstrapIndex.IndexReader indexReader = createBootstrapIndexReader();
diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/CleansCommand.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/CleansCommand.java
index 4924eaacbf46b..de0e4aa109894 100644
--- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/CleansCommand.java
+++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/CleansCommand.java
@@ -32,13 +32,12 @@
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.common.table.timeline.TimelineMetadataUtils;
import org.apache.hudi.utilities.UtilHelpers;
-
import org.apache.spark.launcher.SparkLauncher;
import org.apache.spark.util.Utils;
-import org.springframework.shell.core.CommandMarker;
-import org.springframework.shell.core.annotation.CliCommand;
-import org.springframework.shell.core.annotation.CliOption;
-import org.springframework.stereotype.Component;
+import org.springframework.shell.standard.ShellComponent;
+import org.springframework.shell.standard.ShellMethod;
+import org.springframework.shell.standard.ShellOption;
+import scala.collection.JavaConverters;
import java.io.IOException;
import java.net.URISyntaxException;
@@ -48,21 +47,19 @@
import java.util.Map;
import java.util.stream.Collectors;
-import scala.collection.JavaConverters;
-
/**
* CLI command to show cleans options.
*/
-@Component
-public class CleansCommand implements CommandMarker {
+@ShellComponent
+public class CleansCommand {
- @CliCommand(value = "cleans show", help = "Show the cleans")
+ @ShellMethod(key = "cleans show", value = "Show the cleans")
public String showCleans(
- @CliOption(key = {"limit"}, help = "Limit commits", unspecifiedDefaultValue = "-1") final Integer limit,
- @CliOption(key = {"sortBy"}, help = "Sorting Field", unspecifiedDefaultValue = "") final String sortByField,
- @CliOption(key = {"desc"}, help = "Ordering", unspecifiedDefaultValue = "false") final boolean descending,
- @CliOption(key = {"headeronly"}, help = "Print Header Only",
- unspecifiedDefaultValue = "false") final boolean headerOnly)
+ @ShellOption(value = {"--limit"}, help = "Limit commits", defaultValue = "-1") final Integer limit,
+ @ShellOption(value = {"--sortBy"}, help = "Sorting Field", defaultValue = "") final String sortByField,
+ @ShellOption(value = {"--desc"}, help = "Ordering", defaultValue = "false") final boolean descending,
+ @ShellOption(value = {"--headeronly"}, help = "Print Header Only",
+ defaultValue = "false") final boolean headerOnly)
throws IOException {
HoodieActiveTimeline activeTimeline = HoodieCLI.getTableMetaClient().getActiveTimeline();
@@ -71,9 +68,9 @@ public String showCleans(
List rows = new ArrayList<>();
for (HoodieInstant clean : cleans) {
HoodieCleanMetadata cleanMetadata =
- TimelineMetadataUtils.deserializeHoodieCleanMetadata(timeline.getInstantDetails(clean).get());
- rows.add(new Comparable[]{clean.getTimestamp(), cleanMetadata.getEarliestCommitToRetain(),
- cleanMetadata.getTotalFilesDeleted(), cleanMetadata.getTimeTakenInMillis()});
+ TimelineMetadataUtils.deserializeHoodieCleanMetadata(timeline.getInstantDetails(clean).get());
+ rows.add(new Comparable[] {clean.getTimestamp(), cleanMetadata.getEarliestCommitToRetain(),
+ cleanMetadata.getTotalFilesDeleted(), cleanMetadata.getTimeTakenInMillis()});
}
TableHeader header =
@@ -84,13 +81,14 @@ public String showCleans(
return HoodiePrintHelper.print(header, new HashMap<>(), sortByField, descending, limit, headerOnly, rows);
}
- @CliCommand(value = "clean showpartitions", help = "Show partition level details of a clean")
- public String showCleanPartitions(@CliOption(key = {"clean"}, help = "clean to show") final String instantTime,
- @CliOption(key = {"limit"}, help = "Limit commits", unspecifiedDefaultValue = "-1") final Integer limit,
- @CliOption(key = {"sortBy"}, help = "Sorting Field", unspecifiedDefaultValue = "") final String sortByField,
- @CliOption(key = {"desc"}, help = "Ordering", unspecifiedDefaultValue = "false") final boolean descending,
- @CliOption(key = {"headeronly"}, help = "Print Header Only",
- unspecifiedDefaultValue = "false") final boolean headerOnly)
+ @ShellMethod(key = "clean showpartitions", value = "Show partition level details of a clean")
+ public String showCleanPartitions(
+ @ShellOption(value = {"--clean"}, help = "clean to show") final String instantTime,
+ @ShellOption(value = {"--limit"}, help = "Limit commits", defaultValue = "-1") final Integer limit,
+ @ShellOption(value = {"--sortBy"}, help = "Sorting Field", defaultValue = "") final String sortByField,
+ @ShellOption(value = {"--desc"}, help = "Ordering", defaultValue = "false") final boolean descending,
+ @ShellOption(value = {"--headeronly"}, help = "Print Header Only",
+ defaultValue = "false") final boolean headerOnly)
throws Exception {
HoodieActiveTimeline activeTimeline = HoodieCLI.getTableMetaClient().getActiveTimeline();
@@ -121,14 +119,15 @@ public String showCleanPartitions(@CliOption(key = {"clean"}, help = "clean to s
}
- @CliCommand(value = "cleans run", help = "run clean")
- public String runClean(@CliOption(key = "sparkMemory", unspecifiedDefaultValue = "4G",
- help = "Spark executor memory") final String sparkMemory,
- @CliOption(key = "propsFilePath", help = "path to properties file on localfs or dfs with configurations for hoodie client for cleaning",
- unspecifiedDefaultValue = "") final String propsFilePath,
- @CliOption(key = "hoodieConfigs", help = "Any configuration that can be set in the properties file can be passed here in the form of an array",
- unspecifiedDefaultValue = "") final String[] configs,
- @CliOption(key = "sparkMaster", unspecifiedDefaultValue = "", help = "Spark Master ") String master) throws IOException, InterruptedException, URISyntaxException {
+ @ShellMethod(key = "cleans run", value = "run clean")
+ public String runClean(
+ @ShellOption(value = "--sparkMemory", defaultValue = "4G",
+ help = "Spark executor memory") final String sparkMemory,
+ @ShellOption(value = "--propsFilePath", help = "path to properties file on localfs or dfs with configurations for hoodie client for cleaning",
+ defaultValue = "") final String propsFilePath,
+ @ShellOption(value = "--hoodieConfigs", help = "Any configuration that can be set in the properties file can be passed here in the form of an array",
+ defaultValue = "") final String[] configs,
+ @ShellOption(value = "--sparkMaster", defaultValue = "", help = "Spark Master ") String master) throws IOException, InterruptedException, URISyntaxException {
boolean initialized = HoodieCLI.initConf();
HoodieCLI.initFS(initialized);
HoodieTableMetaClient metaClient = HoodieCLI.getTableMetaClient();
diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/ClusteringCommand.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/ClusteringCommand.java
new file mode 100644
index 0000000000000..963411bf98a1e
--- /dev/null
+++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/ClusteringCommand.java
@@ -0,0 +1,151 @@
+/*
+ * 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.cli.commands;
+
+import org.apache.hudi.cli.HoodieCLI;
+import org.apache.hudi.cli.commands.SparkMain.SparkCommand;
+import org.apache.hudi.cli.utils.InputStreamConsumer;
+import org.apache.hudi.cli.utils.SparkUtil;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
+import org.apache.hudi.utilities.UtilHelpers;
+import org.apache.spark.launcher.SparkLauncher;
+import org.apache.spark.util.Utils;
+import org.springframework.shell.standard.ShellComponent;
+import org.springframework.shell.standard.ShellMethod;
+import org.springframework.shell.standard.ShellOption;
+import scala.collection.JavaConverters;
+
+@ShellComponent
+public class ClusteringCommand {
+
+ /**
+ * Schedule clustering table service.
+ *
+ * Example:
+ * > connect --path {path to hudi table}
+ * > clustering schedule --sparkMaster local --sparkMemory 2g
+ */
+ @ShellMethod(key = "clustering schedule", value = "Schedule Clustering")
+ public String scheduleClustering(
+ @ShellOption(value = "--sparkMaster", defaultValue = SparkUtil.DEFAULT_SPARK_MASTER, help = "Spark master") final String master,
+ @ShellOption(value = "--sparkMemory", defaultValue = "1g", help = "Spark executor memory") final String sparkMemory,
+ @ShellOption(value = "--propsFilePath", help = "path to properties file on localfs or dfs with configurations "
+ + "for hoodie client for clustering", defaultValue = "") final String propsFilePath,
+ @ShellOption(value = "--hoodieConfigs", help = "Any configuration that can be set in the properties file can "
+ + "be passed here in the form of an array", defaultValue = "") final String[] configs) throws Exception {
+ HoodieTableMetaClient client = HoodieCLI.getTableMetaClient();
+ boolean initialized = HoodieCLI.initConf();
+ HoodieCLI.initFS(initialized);
+
+ String sparkPropertiesPath =
+ Utils.getDefaultPropertiesFile(JavaConverters.mapAsScalaMapConverter(System.getenv()).asScala());
+ SparkLauncher sparkLauncher = SparkUtil.initLauncher(sparkPropertiesPath);
+
+ // First get a clustering instant time and pass it to spark launcher for scheduling clustering
+ String clusteringInstantTime = HoodieActiveTimeline.createNewInstantTime();
+
+ sparkLauncher.addAppArgs(SparkCommand.CLUSTERING_SCHEDULE.toString(), master, sparkMemory,
+ client.getBasePath(), client.getTableConfig().getTableName(), clusteringInstantTime, propsFilePath);
+ UtilHelpers.validateAndAddProperties(configs, sparkLauncher);
+ Process process = sparkLauncher.launch();
+ InputStreamConsumer.captureOutput(process);
+ int exitCode = process.waitFor();
+ if (exitCode != 0) {
+ return "Failed to schedule clustering for " + clusteringInstantTime;
+ }
+ return "Succeeded to schedule clustering for " + clusteringInstantTime;
+ }
+
+ /**
+ * Run clustering table service.
+ *
+ * Example:
+ * > connect --path {path to hudi table}
+ * > clustering schedule --sparkMaster local --sparkMemory 2g
+ * > clustering run --sparkMaster local --sparkMemory 2g --clusteringInstant 20211124005208
+ */
+ @ShellMethod(key = "clustering run", value = "Run Clustering")
+ public String runClustering(
+ @ShellOption(value = "--sparkMaster", defaultValue = SparkUtil.DEFAULT_SPARK_MASTER, help = "Spark master") final String master,
+ @ShellOption(value = "--sparkMemory", help = "Spark executor memory", defaultValue = "4g") final String sparkMemory,
+ @ShellOption(value = "--parallelism", help = "Parallelism for hoodie clustering", defaultValue = "1") final String parallelism,
+ @ShellOption(value = "--retry", help = "Number of retries", defaultValue = "1") final String retry,
+ @ShellOption(value = "--clusteringInstant", help = "Clustering instant time",
+ defaultValue = ShellOption.NULL) final String clusteringInstantTime,
+ @ShellOption(value = "--propsFilePath", help = "path to properties file on localfs or dfs with configurations for "
+ + "hoodie client for compacting", defaultValue = "") final String propsFilePath,
+ @ShellOption(value = "--hoodieConfigs", help = "Any configuration that can be set in the properties file can be "
+ + "passed here in the form of an array", defaultValue = "") final String[] configs) throws Exception {
+ HoodieTableMetaClient client = HoodieCLI.getTableMetaClient();
+ boolean initialized = HoodieCLI.initConf();
+ HoodieCLI.initFS(initialized);
+
+ String sparkPropertiesPath =
+ Utils.getDefaultPropertiesFile(JavaConverters.mapAsScalaMapConverter(System.getenv()).asScala());
+ SparkLauncher sparkLauncher = SparkUtil.initLauncher(sparkPropertiesPath);
+ sparkLauncher.addAppArgs(SparkCommand.CLUSTERING_RUN.toString(), master, sparkMemory,
+ client.getBasePath(), client.getTableConfig().getTableName(), clusteringInstantTime,
+ parallelism, retry, propsFilePath);
+ UtilHelpers.validateAndAddProperties(configs, sparkLauncher);
+ Process process = sparkLauncher.launch();
+ InputStreamConsumer.captureOutput(process);
+ int exitCode = process.waitFor();
+ if (exitCode != 0) {
+ return "Failed to run clustering for " + clusteringInstantTime;
+ }
+ return "Succeeded to run clustering for " + clusteringInstantTime;
+ }
+
+ /**
+ * Run clustering table service.
+ *
+ * Example:
+ * > connect --path {path to hudi table}
+ * > clustering scheduleAndExecute --sparkMaster local --sparkMemory 2g
+ */
+ @ShellMethod(key = "clustering scheduleAndExecute", value = "Run Clustering. Make a cluster plan first and execute that plan immediately")
+ public String runClustering(
+ @ShellOption(value = "--sparkMaster", defaultValue = SparkUtil.DEFAULT_SPARK_MASTER, help = "Spark master") final String master,
+ @ShellOption(value = "--sparkMemory", help = "Spark executor memory", defaultValue = "4g") final String sparkMemory,
+ @ShellOption(value = "--parallelism", help = "Parallelism for hoodie clustering", defaultValue = "1") final String parallelism,
+ @ShellOption(value = "--retry", help = "Number of retries", defaultValue = "1") final String retry,
+ @ShellOption(value = "--propsFilePath", help = "path to properties file on localfs or dfs with configurations for "
+ + "hoodie client for compacting", defaultValue = "") final String propsFilePath,
+ @ShellOption(value = "--hoodieConfigs", help = "Any configuration that can be set in the properties file can be "
+ + "passed here in the form of an array", defaultValue = "") final String[] configs) throws Exception {
+ HoodieTableMetaClient client = HoodieCLI.getTableMetaClient();
+ boolean initialized = HoodieCLI.initConf();
+ HoodieCLI.initFS(initialized);
+
+ String sparkPropertiesPath =
+ Utils.getDefaultPropertiesFile(JavaConverters.mapAsScalaMapConverter(System.getenv()).asScala());
+ SparkLauncher sparkLauncher = SparkUtil.initLauncher(sparkPropertiesPath);
+ sparkLauncher.addAppArgs(SparkCommand.CLUSTERING_SCHEDULE_AND_EXECUTE.toString(), master, sparkMemory,
+ client.getBasePath(), client.getTableConfig().getTableName(), parallelism, retry, propsFilePath);
+ UtilHelpers.validateAndAddProperties(configs, sparkLauncher);
+ Process process = sparkLauncher.launch();
+ InputStreamConsumer.captureOutput(process);
+ int exitCode = process.waitFor();
+ if (exitCode != 0) {
+ return "Failed to run clustering for scheduleAndExecute.";
+ }
+ return "Succeeded to run clustering for scheduleAndExecute";
+ }
+}
diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/CommitsCommand.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/CommitsCommand.java
index 852a413b014ac..e269f8da0cba8 100644
--- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/CommitsCommand.java
+++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/CommitsCommand.java
@@ -22,200 +22,176 @@
import org.apache.hudi.cli.HoodiePrintHelper;
import org.apache.hudi.cli.HoodieTableHeaderFields;
import org.apache.hudi.cli.TableHeader;
-import org.apache.hudi.cli.utils.CommitUtil;
-import org.apache.hudi.cli.utils.InputStreamConsumer;
-import org.apache.hudi.cli.utils.SparkUtil;
import org.apache.hudi.common.model.HoodieCommitMetadata;
+import org.apache.hudi.common.model.HoodieReplaceCommitMetadata;
import org.apache.hudi.common.model.HoodieWriteStat;
import org.apache.hudi.common.table.HoodieTableMetaClient;
-import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
import org.apache.hudi.common.table.timeline.HoodieArchivedTimeline;
import org.apache.hudi.common.table.timeline.HoodieDefaultTimeline;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.common.util.NumericUtils;
+import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.StringUtils;
-import org.apache.spark.launcher.SparkLauncher;
-import org.springframework.shell.core.CommandMarker;
-import org.springframework.shell.core.annotation.CliCommand;
-import org.springframework.shell.core.annotation.CliOption;
-import org.springframework.stereotype.Component;
+import org.springframework.shell.standard.ShellComponent;
+import org.springframework.shell.standard.ShellMethod;
+import org.springframework.shell.standard.ShellOption;
import java.io.IOException;
import java.util.ArrayList;
-import java.util.Collections;
+import java.util.Arrays;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.function.Function;
import java.util.stream.Collectors;
+import static org.apache.hudi.cli.utils.CommitUtil.getTimeDaysAgo;
+import static org.apache.hudi.common.table.timeline.TimelineUtils.getTimeline;
+
/**
* CLI command to display commits options.
*/
-@Component
-public class CommitsCommand implements CommandMarker {
+@ShellComponent
+public class CommitsCommand {
private String printCommits(HoodieDefaultTimeline timeline,
- final Integer limit, final String sortByField,
+ final Integer limit,
+ final String sortByField,
final boolean descending,
final boolean headerOnly,
final String tempTableName) throws IOException {
final List rows = new ArrayList<>();
final List commits = timeline.getCommitsTimeline().filterCompletedInstants()
- .getInstants().collect(Collectors.toList());
- // timeline can be read from multiple files. So sort is needed instead of reversing the collection
- Collections.sort(commits, HoodieInstant.COMPARATOR.reversed());
-
- for (int i = 0; i < commits.size(); i++) {
- final HoodieInstant commit = commits.get(i);
- final HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes(
- timeline.getInstantDetails(commit).get(),
- HoodieCommitMetadata.class);
- rows.add(new Comparable[]{commit.getTimestamp(),
- commitMetadata.fetchTotalBytesWritten(),
- commitMetadata.fetchTotalFilesInsert(),
- commitMetadata.fetchTotalFilesUpdated(),
- commitMetadata.fetchTotalPartitionsWritten(),
- commitMetadata.fetchTotalRecordsWritten(),
- commitMetadata.fetchTotalUpdateRecordsWritten(),
- commitMetadata.fetchTotalWriteErrors()});
+ .getInstants().sorted(HoodieInstant.COMPARATOR.reversed()).collect(Collectors.toList());
+
+ for (final HoodieInstant commit : commits) {
+ if (timeline.getInstantDetails(commit).isPresent()) {
+ final HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes(
+ timeline.getInstantDetails(commit).get(),
+ HoodieCommitMetadata.class);
+ rows.add(new Comparable[] {commit.getTimestamp(),
+ commitMetadata.fetchTotalBytesWritten(),
+ commitMetadata.fetchTotalFilesInsert(),
+ commitMetadata.fetchTotalFilesUpdated(),
+ commitMetadata.fetchTotalPartitionsWritten(),
+ commitMetadata.fetchTotalRecordsWritten(),
+ commitMetadata.fetchTotalUpdateRecordsWritten(),
+ commitMetadata.fetchTotalWriteErrors()});
+ }
}
final Map> fieldNameToConverterMap = new HashMap<>();
- fieldNameToConverterMap.put(HoodieTableHeaderFields.HEADER_TOTAL_BYTES_WRITTEN, entry -> {
- return NumericUtils.humanReadableByteCount((Double.valueOf(entry.toString())));
- });
-
- final TableHeader header = new TableHeader()
- .addTableHeaderField(HoodieTableHeaderFields.HEADER_COMMIT_TIME)
- .addTableHeaderField(HoodieTableHeaderFields.HEADER_TOTAL_BYTES_WRITTEN)
- .addTableHeaderField(HoodieTableHeaderFields.HEADER_TOTAL_FILES_ADDED)
- .addTableHeaderField(HoodieTableHeaderFields.HEADER_TOTAL_FILES_UPDATED)
- .addTableHeaderField(HoodieTableHeaderFields.HEADER_TOTAL_PARTITIONS_WRITTEN)
- .addTableHeaderField(HoodieTableHeaderFields.HEADER_TOTAL_RECORDS_WRITTEN)
- .addTableHeaderField(HoodieTableHeaderFields.HEADER_TOTAL_UPDATE_RECORDS_WRITTEN)
- .addTableHeaderField(HoodieTableHeaderFields.HEADER_TOTAL_ERRORS);
+ fieldNameToConverterMap.put(
+ HoodieTableHeaderFields.HEADER_TOTAL_BYTES_WRITTEN,
+ entry -> NumericUtils.humanReadableByteCount((Double.parseDouble(entry.toString()))));
+
+ final TableHeader header = HoodieTableHeaderFields.getTableHeader();
return HoodiePrintHelper.print(header, fieldNameToConverterMap, sortByField, descending,
- limit, headerOnly, rows, tempTableName);
+ limit, headerOnly, rows, tempTableName);
}
private String printCommitsWithMetadata(HoodieDefaultTimeline timeline,
- final Integer limit, final String sortByField,
- final boolean descending,
- final boolean headerOnly,
- final String tempTableName) throws IOException {
+ final Integer limit, final String sortByField,
+ final boolean descending,
+ final boolean headerOnly,
+ final String tempTableName,
+ final String partition) throws IOException {
final List rows = new ArrayList<>();
final List commits = timeline.getCommitsTimeline().filterCompletedInstants()
- .getInstants().collect(Collectors.toList());
- // timeline can be read from multiple files. So sort is needed instead of reversing the collection
- Collections.sort(commits, HoodieInstant.COMPARATOR.reversed());
-
- for (int i = 0; i < commits.size(); i++) {
- final HoodieInstant commit = commits.get(i);
- final HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes(
- timeline.getInstantDetails(commit).get(),
- HoodieCommitMetadata.class);
-
- for (Map.Entry> partitionWriteStat :
- commitMetadata.getPartitionToWriteStats().entrySet()) {
- for (HoodieWriteStat hoodieWriteStat : partitionWriteStat.getValue()) {
- rows.add(new Comparable[]{ commit.getAction(), commit.getTimestamp(), hoodieWriteStat.getPartitionPath(),
+ .getInstants().sorted(HoodieInstant.COMPARATOR.reversed()).collect(Collectors.toList());
+
+ for (final HoodieInstant commit : commits) {
+ if (timeline.getInstantDetails(commit).isPresent()) {
+ final HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes(
+ timeline.getInstantDetails(commit).get(),
+ HoodieCommitMetadata.class);
+
+ for (Map.Entry> partitionWriteStat :
+ commitMetadata.getPartitionToWriteStats().entrySet()) {
+ for (HoodieWriteStat hoodieWriteStat : partitionWriteStat.getValue()) {
+ if (StringUtils.isNullOrEmpty(partition) || partition.equals(hoodieWriteStat.getPartitionPath())) {
+ rows.add(new Comparable[] {commit.getAction(), commit.getTimestamp(), hoodieWriteStat.getPartitionPath(),
hoodieWriteStat.getFileId(), hoodieWriteStat.getPrevCommit(), hoodieWriteStat.getNumWrites(),
hoodieWriteStat.getNumInserts(), hoodieWriteStat.getNumDeletes(),
hoodieWriteStat.getNumUpdateWrites(), hoodieWriteStat.getTotalWriteErrors(),
hoodieWriteStat.getTotalLogBlocks(), hoodieWriteStat.getTotalCorruptLogBlock(),
hoodieWriteStat.getTotalRollbackBlocks(), hoodieWriteStat.getTotalLogRecords(),
hoodieWriteStat.getTotalUpdatedRecordsCompacted(), hoodieWriteStat.getTotalWriteBytes()
- });
+ });
+ }
+ }
}
}
}
final Map> fieldNameToConverterMap = new HashMap<>();
- fieldNameToConverterMap.put(HoodieTableHeaderFields.HEADER_TOTAL_BYTES_WRITTEN, entry -> {
- return NumericUtils.humanReadableByteCount((Double.valueOf(entry.toString())));
- });
-
- TableHeader header = new TableHeader().addTableHeaderField(HoodieTableHeaderFields.HEADER_ACTION)
- .addTableHeaderField(HoodieTableHeaderFields.HEADER_INSTANT)
- .addTableHeaderField(HoodieTableHeaderFields.HEADER_PARTITION)
- .addTableHeaderField(HoodieTableHeaderFields.HEADER_FILE_ID)
- .addTableHeaderField(HoodieTableHeaderFields.HEADER_PREVIOUS_COMMIT)
- .addTableHeaderField(HoodieTableHeaderFields.HEADER_NUM_WRITES)
- .addTableHeaderField(HoodieTableHeaderFields.HEADER_NUM_INSERTS)
- .addTableHeaderField(HoodieTableHeaderFields.HEADER_NUM_DELETES)
- .addTableHeaderField(HoodieTableHeaderFields.HEADER_NUM_UPDATE_WRITES)
- .addTableHeaderField(HoodieTableHeaderFields.HEADER_TOTAL_ERRORS)
- .addTableHeaderField(HoodieTableHeaderFields.HEADER_TOTAL_LOG_BLOCKS)
- .addTableHeaderField(HoodieTableHeaderFields.HEADER_TOTAL_CORRUPT_LOG_BLOCKS)
- .addTableHeaderField(HoodieTableHeaderFields.HEADER_TOTAL_ROLLBACK_BLOCKS)
- .addTableHeaderField(HoodieTableHeaderFields.HEADER_TOTAL_LOG_RECORDS)
- .addTableHeaderField(HoodieTableHeaderFields.HEADER_TOTAL_UPDATED_RECORDS_COMPACTED)
- .addTableHeaderField(HoodieTableHeaderFields.HEADER_TOTAL_BYTES_WRITTEN);
+ fieldNameToConverterMap.put(
+ HoodieTableHeaderFields.HEADER_TOTAL_BYTES_WRITTEN,
+ entry -> NumericUtils.humanReadableByteCount((Double.parseDouble(entry.toString()))));
- return HoodiePrintHelper.print(header, new HashMap<>(), sortByField, descending,
- limit, headerOnly, rows, tempTableName);
+ return HoodiePrintHelper.print(HoodieTableHeaderFields.getTableHeaderWithExtraMetadata(),
+ fieldNameToConverterMap, sortByField, descending, limit, headerOnly, rows, tempTableName);
}
- @CliCommand(value = "commits show", help = "Show the commits")
+ @ShellMethod(key = "commits show", value = "Show the commits")
public String showCommits(
- @CliOption(key = {"includeExtraMetadata"}, help = "Include extra metadata",
- unspecifiedDefaultValue = "false") final boolean includeExtraMetadata,
- @CliOption(key = {"createView"}, mandatory = false, help = "view name to store output table",
- unspecifiedDefaultValue = "") final String exportTableName,
- @CliOption(key = {"limit"}, help = "Limit commits",
- unspecifiedDefaultValue = "-1") final Integer limit,
- @CliOption(key = {"sortBy"}, help = "Sorting Field", unspecifiedDefaultValue = "") final String sortByField,
- @CliOption(key = {"desc"}, help = "Ordering", unspecifiedDefaultValue = "false") final boolean descending,
- @CliOption(key = {"headeronly"}, help = "Print Header Only",
- unspecifiedDefaultValue = "false") final boolean headerOnly)
+ @ShellOption(value = {"--includeExtraMetadata"}, help = "Include extra metadata",
+ defaultValue = "false") final boolean includeExtraMetadata,
+ @ShellOption(value = {"--createView"}, help = "view name to store output table",
+ defaultValue = "") final String exportTableName,
+ @ShellOption(value = {"--limit"}, help = "Limit commits",
+ defaultValue = "-1") final Integer limit,
+ @ShellOption(value = {"--sortBy"}, help = "Sorting Field", defaultValue = "") final String sortByField,
+ @ShellOption(value = {"--desc"}, help = "Ordering", defaultValue = "false") final boolean descending,
+ @ShellOption(value = {"--headeronly"}, help = "Print Header Only",
+ defaultValue = "false") final boolean headerOnly,
+ @ShellOption(value = {"--partition"}, help = "Partition value", defaultValue = ShellOption.NULL) final String partition,
+ @ShellOption(value = {"--includeArchivedTimeline"}, help = "Include archived commits as well",
+ defaultValue = "false") final boolean includeArchivedTimeline)
throws IOException {
- HoodieActiveTimeline activeTimeline = HoodieCLI.getTableMetaClient().getActiveTimeline();
+ HoodieDefaultTimeline timeline = getTimeline(HoodieCLI.getTableMetaClient(), includeArchivedTimeline);
if (includeExtraMetadata) {
- return printCommitsWithMetadata(activeTimeline, limit, sortByField, descending, headerOnly, exportTableName);
- } else {
- return printCommits(activeTimeline, limit, sortByField, descending, headerOnly, exportTableName);
+ return printCommitsWithMetadata(timeline, limit, sortByField, descending, headerOnly, exportTableName, partition);
+ } else {
+ return printCommits(timeline, limit, sortByField, descending, headerOnly, exportTableName);
}
}
- @CliCommand(value = "commits showarchived", help = "Show the archived commits")
+ @ShellMethod(key = "commits showarchived", value = "Show the archived commits")
public String showArchivedCommits(
- @CliOption(key = {"includeExtraMetadata"}, help = "Include extra metadata",
- unspecifiedDefaultValue = "false") final boolean includeExtraMetadata,
- @CliOption(key = {"createView"}, mandatory = false, help = "view name to store output table",
- unspecifiedDefaultValue = "") final String exportTableName,
- @CliOption(key = {"startTs"}, mandatory = false, help = "start time for commits, default: now - 10 days")
+ @ShellOption(value = {"--includeExtraMetadata"}, help = "Include extra metadata",
+ defaultValue = "false") final boolean includeExtraMetadata,
+ @ShellOption(value = {"--createView"}, help = "view name to store output table",
+ defaultValue = "") final String exportTableName,
+ @ShellOption(value = {"--startTs"}, defaultValue = ShellOption.NULL, help = "start time for commits, default: now - 10 days")
String startTs,
- @CliOption(key = {"endTs"}, mandatory = false, help = "end time for commits, default: now - 1 day")
+ @ShellOption(value = {"--endTs"}, defaultValue = ShellOption.NULL, help = "end time for commits, default: now - 1 day")
String endTs,
- @CliOption(key = {"limit"}, mandatory = false, help = "Limit commits", unspecifiedDefaultValue = "-1")
- final Integer limit,
- @CliOption(key = {"sortBy"}, help = "Sorting Field", unspecifiedDefaultValue = "")
- final String sortByField,
- @CliOption(key = {"desc"}, help = "Ordering", unspecifiedDefaultValue = "false")
- final boolean descending,
- @CliOption(key = {"headeronly"}, help = "Print Header Only", unspecifiedDefaultValue = "false")
- final boolean headerOnly)
- throws IOException {
+ @ShellOption(value = {"--limit"}, help = "Limit commits", defaultValue = "-1") final Integer limit,
+ @ShellOption(value = {"--sortBy"}, help = "Sorting Field", defaultValue = "") final String sortByField,
+ @ShellOption(value = {"--desc"}, help = "Ordering", defaultValue = "false") final boolean descending,
+ @ShellOption(value = {"--headeronly"}, help = "Print Header Only", defaultValue = "false") final boolean headerOnly,
+ @ShellOption(value = {"--partition"}, help = "Partition value", defaultValue = ShellOption.NULL) final String partition)
+ throws IOException {
if (StringUtils.isNullOrEmpty(startTs)) {
- startTs = CommitUtil.getTimeDaysAgo(10);
+ startTs = getTimeDaysAgo(10);
}
if (StringUtils.isNullOrEmpty(endTs)) {
- endTs = CommitUtil.getTimeDaysAgo(1);
+ endTs = getTimeDaysAgo(1);
}
HoodieArchivedTimeline archivedTimeline = HoodieCLI.getTableMetaClient().getArchivedTimeline();
try {
archivedTimeline.loadInstantDetailsInMemory(startTs, endTs);
HoodieDefaultTimeline timelineRange = archivedTimeline.findInstantsInRange(startTs, endTs);
if (includeExtraMetadata) {
- return printCommitsWithMetadata(timelineRange, limit, sortByField, descending, headerOnly, exportTableName);
- } else {
+ return printCommitsWithMetadata(timelineRange, limit, sortByField, descending, headerOnly, exportTableName, partition);
+ } else {
return printCommits(timelineRange, limit, sortByField, descending, headerOnly, exportTableName);
}
} finally {
@@ -224,57 +200,34 @@ public String showArchivedCommits(
}
}
- @CliCommand(value = "commit rollback", help = "Rollback a commit")
- public String rollbackCommit(@CliOption(key = {"commit"}, help = "Commit to rollback") final String instantTime,
- @CliOption(key = {"sparkProperties"}, help = "Spark Properties File Path") final String sparkPropertiesPath,
- @CliOption(key = "sparkMaster", unspecifiedDefaultValue = "", help = "Spark Master") String master,
- @CliOption(key = "sparkMemory", unspecifiedDefaultValue = "4G",
- help = "Spark executor memory") final String sparkMemory)
- throws Exception {
- HoodieActiveTimeline activeTimeline = HoodieCLI.getTableMetaClient().getActiveTimeline();
- HoodieTimeline completedTimeline = activeTimeline.getCommitsTimeline().filterCompletedInstants();
- HoodieTimeline filteredTimeline = completedTimeline.filter(instant -> instant.getTimestamp().equals(instantTime));
- if (filteredTimeline.empty()) {
- return "Commit " + instantTime + " not found in Commits " + completedTimeline;
- }
-
- SparkLauncher sparkLauncher = SparkUtil.initLauncher(sparkPropertiesPath);
- sparkLauncher.addAppArgs(SparkMain.SparkCommand.ROLLBACK.toString(), master, sparkMemory, instantTime,
- HoodieCLI.getTableMetaClient().getBasePath());
- Process process = sparkLauncher.launch();
- InputStreamConsumer.captureOutput(process);
- int exitCode = process.waitFor();
- // Refresh the current
- HoodieCLI.refreshTableMetadata();
- if (exitCode != 0) {
- return "Commit " + instantTime + " failed to roll back";
- }
- return "Commit " + instantTime + " rolled back";
- }
-
- @CliCommand(value = "commit showpartitions", help = "Show partition level details of a commit")
+ @ShellMethod(key = "commit showpartitions", value = "Show partition level details of a commit")
public String showCommitPartitions(
- @CliOption(key = {"createView"}, mandatory = false, help = "view name to store output table",
- unspecifiedDefaultValue = "") final String exportTableName,
- @CliOption(key = {"commit"}, help = "Commit to show") final String instantTime,
- @CliOption(key = {"limit"}, help = "Limit commits", unspecifiedDefaultValue = "-1") final Integer limit,
- @CliOption(key = {"sortBy"}, help = "Sorting Field", unspecifiedDefaultValue = "") final String sortByField,
- @CliOption(key = {"desc"}, help = "Ordering", unspecifiedDefaultValue = "false") final boolean descending,
- @CliOption(key = {"headeronly"}, help = "Print Header Only",
- unspecifiedDefaultValue = "false") final boolean headerOnly)
+ @ShellOption(value = {"--createView"}, help = "view name to store output table",
+ defaultValue = "") final String exportTableName,
+ @ShellOption(value = {"--commit"}, help = "Commit to show") final String instantTime,
+ @ShellOption(value = {"--limit"}, help = "Limit commits", defaultValue = "-1") final Integer limit,
+ @ShellOption(value = {"--sortBy"}, help = "Sorting Field", defaultValue = "") final String sortByField,
+ @ShellOption(value = {"--desc"}, help = "Ordering", defaultValue = "false") final boolean descending,
+ @ShellOption(value = {"--headeronly"}, help = "Print Header Only",
+ defaultValue = "false") final boolean headerOnly,
+ @ShellOption(value = {"includeArchivedTimeline"}, help = "Include archived commits as well",
+ defaultValue = "false") final boolean includeArchivedTimeline)
throws Exception {
- HoodieActiveTimeline activeTimeline = HoodieCLI.getTableMetaClient().getActiveTimeline();
- HoodieTimeline timeline = activeTimeline.getCommitsTimeline().filterCompletedInstants();
- HoodieInstant commitInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, instantTime);
+ HoodieDefaultTimeline defaultTimeline = getTimeline(HoodieCLI.getTableMetaClient(), includeArchivedTimeline);
+ HoodieTimeline timeline = defaultTimeline.getCommitsTimeline().filterCompletedInstants();
+
+ Option hoodieInstantOption = getCommitForInstant(timeline, instantTime);
+ Option commitMetadataOptional = getHoodieCommitMetadata(timeline, hoodieInstantOption);
- if (!timeline.containsInstant(commitInstant)) {
+ if (!commitMetadataOptional.isPresent()) {
return "Commit " + instantTime + " not found in Commits " + timeline;
}
- HoodieCommitMetadata meta = HoodieCommitMetadata.fromBytes(activeTimeline.getInstantDetails(commitInstant).get(),
- HoodieCommitMetadata.class);
+
+ HoodieCommitMetadata meta = commitMetadataOptional.get();
List rows = new ArrayList<>();
for (Map.Entry> entry : meta.getPartitionToWriteStats().entrySet()) {
+ String action = hoodieInstantOption.get().getAction();
String path = entry.getKey();
List stats = entry.getValue();
long totalFilesAdded = 0;
@@ -294,7 +247,7 @@ public String showCommitPartitions(
totalBytesWritten += stat.getTotalWriteBytes();
totalWriteErrors += stat.getTotalWriteErrors();
}
- rows.add(new Comparable[] {path, totalFilesAdded, totalFilesUpdated, totalRecordsInserted, totalRecordsUpdated,
+ rows.add(new Comparable[] {action, path, totalFilesAdded, totalFilesUpdated, totalRecordsInserted, totalRecordsUpdated,
totalBytesWritten, totalWriteErrors});
}
@@ -302,7 +255,8 @@ public String showCommitPartitions(
fieldNameToConverterMap.put(HoodieTableHeaderFields.HEADER_TOTAL_BYTES_WRITTEN, entry ->
NumericUtils.humanReadableByteCount((Long.parseLong(entry.toString()))));
- TableHeader header = new TableHeader().addTableHeaderField(HoodieTableHeaderFields.HEADER_PARTITION_PATH)
+ TableHeader header = new TableHeader().addTableHeaderField(HoodieTableHeaderFields.HEADER_ACTION)
+ .addTableHeaderField(HoodieTableHeaderFields.HEADER_PARTITION_PATH)
.addTableHeaderField(HoodieTableHeaderFields.HEADER_TOTAL_FILES_ADDED)
.addTableHeaderField(HoodieTableHeaderFields.HEADER_TOTAL_FILES_UPDATED)
.addTableHeaderField(HoodieTableHeaderFields.HEADER_TOTAL_RECORDS_INSERTED)
@@ -314,38 +268,90 @@ public String showCommitPartitions(
limit, headerOnly, rows, exportTableName);
}
- @CliCommand(value = "commit showfiles", help = "Show file level details of a commit")
+ @ShellMethod(key = "commit show_write_stats", value = "Show write stats of a commit")
+ public String showWriteStats(
+ @ShellOption(value = {"--createView"}, help = "view name to store output table",
+ defaultValue = "") final String exportTableName,
+ @ShellOption(value = {"--commit"}, help = "Commit to show") final String instantTime,
+ @ShellOption(value = {"--limit"}, help = "Limit commits", defaultValue = "-1") final Integer limit,
+ @ShellOption(value = {"--sortBy"}, help = "Sorting Field", defaultValue = "") final String sortByField,
+ @ShellOption(value = {"--desc"}, help = "Ordering", defaultValue = "false") final boolean descending,
+ @ShellOption(value = {"--headeronly"}, help = "Print Header Only",
+ defaultValue = "false") final boolean headerOnly,
+ @ShellOption(value = {"includeArchivedTimeline"}, help = "Include archived commits as well",
+ defaultValue = "false") final boolean includeArchivedTimeline)
+ throws Exception {
+
+ HoodieDefaultTimeline defaultTimeline = getTimeline(HoodieCLI.getTableMetaClient(), includeArchivedTimeline);
+ HoodieTimeline timeline = defaultTimeline.getCommitsTimeline().filterCompletedInstants();
+
+ Option hoodieInstantOption = getCommitForInstant(timeline, instantTime);
+ Option commitMetadataOptional = getHoodieCommitMetadata(timeline, hoodieInstantOption);
+
+ if (!commitMetadataOptional.isPresent()) {
+ return "Commit " + instantTime + " not found in Commits " + timeline;
+ }
+
+ HoodieCommitMetadata meta = commitMetadataOptional.get();
+
+ String action = hoodieInstantOption.get().getAction();
+ long recordsWritten = meta.fetchTotalRecordsWritten();
+ long bytesWritten = meta.fetchTotalBytesWritten();
+ long avgRecSize = (long) Math.ceil((1.0 * bytesWritten) / recordsWritten);
+ List rows = new ArrayList<>();
+ rows.add(new Comparable[] {action, bytesWritten, recordsWritten, avgRecSize});
+
+ Map> fieldNameToConverterMap = new HashMap<>();
+ fieldNameToConverterMap.put(HoodieTableHeaderFields.HEADER_TOTAL_BYTES_WRITTEN, entry ->
+ NumericUtils.humanReadableByteCount((Long.parseLong(entry.toString()))));
+
+ TableHeader header = new TableHeader().addTableHeaderField(HoodieTableHeaderFields.HEADER_ACTION)
+ .addTableHeaderField(HoodieTableHeaderFields.HEADER_TOTAL_BYTES_WRITTEN_COMMIT)
+ .addTableHeaderField(HoodieTableHeaderFields.HEADER_TOTAL_RECORDS_WRITTEN_COMMIT)
+ .addTableHeaderField(HoodieTableHeaderFields.HEADER_AVG_REC_SIZE_COMMIT);
+
+ return HoodiePrintHelper.print(header, fieldNameToConverterMap, sortByField, descending,
+ limit, headerOnly, rows, exportTableName);
+ }
+
+ @ShellMethod(key = "commit showfiles", value = "Show file level details of a commit")
public String showCommitFiles(
- @CliOption(key = {"createView"}, mandatory = false, help = "view name to store output table",
- unspecifiedDefaultValue = "") final String exportTableName,
- @CliOption(key = {"commit"}, help = "Commit to show") final String instantTime,
- @CliOption(key = {"limit"}, help = "Limit commits", unspecifiedDefaultValue = "-1") final Integer limit,
- @CliOption(key = {"sortBy"}, help = "Sorting Field", unspecifiedDefaultValue = "") final String sortByField,
- @CliOption(key = {"desc"}, help = "Ordering", unspecifiedDefaultValue = "false") final boolean descending,
- @CliOption(key = {"headeronly"}, help = "Print Header Only",
- unspecifiedDefaultValue = "false") final boolean headerOnly)
+ @ShellOption(value = {"--createView"}, help = "view name to store output table",
+ defaultValue = "") final String exportTableName,
+ @ShellOption(value = {"--commit"}, help = "Commit to show") final String instantTime,
+ @ShellOption(value = {"--limit"}, help = "Limit commits", defaultValue = "-1") final Integer limit,
+ @ShellOption(value = {"--sortBy"}, help = "Sorting Field", defaultValue = "") final String sortByField,
+ @ShellOption(value = {"--desc"}, help = "Ordering", defaultValue = "false") final boolean descending,
+ @ShellOption(value = {"--headeronly"}, help = "Print Header Only",
+ defaultValue = "false") final boolean headerOnly,
+ @ShellOption(value = {"includeArchivedTimeline"}, help = "Include archived commits as well",
+ defaultValue = "false") final boolean includeArchivedTimeline)
throws Exception {
- HoodieActiveTimeline activeTimeline = HoodieCLI.getTableMetaClient().getActiveTimeline();
- HoodieTimeline timeline = activeTimeline.getCommitsTimeline().filterCompletedInstants();
- HoodieInstant commitInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, instantTime);
+ HoodieDefaultTimeline defaultTimeline = getTimeline(HoodieCLI.getTableMetaClient(), includeArchivedTimeline);
+ HoodieTimeline timeline = defaultTimeline.getCommitsTimeline().filterCompletedInstants();
- if (!timeline.containsInstant(commitInstant)) {
+ Option hoodieInstantOption = getCommitForInstant(timeline, instantTime);
+ Option commitMetadataOptional = getHoodieCommitMetadata(timeline, hoodieInstantOption);
+
+ if (!commitMetadataOptional.isPresent()) {
return "Commit " + instantTime + " not found in Commits " + timeline;
}
- HoodieCommitMetadata meta = HoodieCommitMetadata.fromBytes(activeTimeline.getInstantDetails(commitInstant).get(),
- HoodieCommitMetadata.class);
+
+ HoodieCommitMetadata meta = commitMetadataOptional.get();
List rows = new ArrayList<>();
for (Map.Entry> entry : meta.getPartitionToWriteStats().entrySet()) {
+ String action = hoodieInstantOption.get().getAction();
String path = entry.getKey();
List stats = entry.getValue();
for (HoodieWriteStat stat : stats) {
- rows.add(new Comparable[] {path, stat.getFileId(), stat.getPrevCommit(), stat.getNumUpdateWrites(),
+ rows.add(new Comparable[] {action, path, stat.getFileId(), stat.getPrevCommit(), stat.getNumUpdateWrites(),
stat.getNumWrites(), stat.getTotalWriteBytes(), stat.getTotalWriteErrors(), stat.getFileSizeInBytes()});
}
}
- TableHeader header = new TableHeader().addTableHeaderField(HoodieTableHeaderFields.HEADER_PARTITION_PATH)
+ TableHeader header = new TableHeader().addTableHeaderField(HoodieTableHeaderFields.HEADER_ACTION)
+ .addTableHeaderField(HoodieTableHeaderFields.HEADER_PARTITION_PATH)
.addTableHeaderField(HoodieTableHeaderFields.HEADER_FILE_ID)
.addTableHeaderField(HoodieTableHeaderFields.HEADER_PREVIOUS_COMMIT)
.addTableHeaderField(HoodieTableHeaderFields.HEADER_TOTAL_RECORDS_UPDATED)
@@ -358,11 +364,11 @@ public String showCommitFiles(
limit, headerOnly, rows, exportTableName);
}
- @CliCommand(value = "commits compare", help = "Compare commits with another Hoodie table")
- public String compareCommits(@CliOption(key = {"path"}, help = "Path of the table to compare to") final String path) {
+ @ShellMethod(key = "commits compare", value = "Compare commits with another Hoodie table")
+ public String compareCommits(@ShellOption(value = {"--path"}, help = "Path of the table to compare to") final String path) {
HoodieTableMetaClient source = HoodieCLI.getTableMetaClient();
- HoodieTableMetaClient target = new HoodieTableMetaClient(HoodieCLI.conf, path);
+ HoodieTableMetaClient target = HoodieTableMetaClient.builder().setConf(HoodieCLI.conf).setBasePath(path).build();
HoodieTimeline targetTimeline = target.getActiveTimeline().getCommitsTimeline().filterCompletedInstants();
HoodieTimeline sourceTimeline = source.getActiveTimeline().getCommitsTimeline().filterCompletedInstants();
String targetLatestCommit =
@@ -385,11 +391,36 @@ public String compareCommits(@CliOption(key = {"path"}, help = "Path of the tabl
}
}
- @CliCommand(value = "commits sync", help = "Compare commits with another Hoodie table")
- public String syncCommits(@CliOption(key = {"path"}, help = "Path of the table to compare to") final String path) {
- HoodieCLI.syncTableMetadata = new HoodieTableMetaClient(HoodieCLI.conf, path);
+ @ShellMethod(key = "commits sync", value = "Sync commits with another Hoodie table")
+ public String syncCommits(@ShellOption(value = {"--path"}, help = "Path of the table to sync to") final String path) {
+ HoodieCLI.syncTableMetadata = HoodieTableMetaClient.builder().setConf(HoodieCLI.conf).setBasePath(path).build();
HoodieCLI.state = HoodieCLI.CLIState.SYNC;
return "Load sync state between " + HoodieCLI.getTableMetaClient().getTableConfig().getTableName() + " and "
+ HoodieCLI.syncTableMetadata.getTableConfig().getTableName();
}
+
+ /*
+ Checks whether a commit or replacecommit action exists in the timeline.
+ * */
+ private Option getCommitForInstant(HoodieTimeline timeline, String instantTime) {
+ List instants = Arrays.asList(
+ new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, instantTime),
+ new HoodieInstant(false, HoodieTimeline.REPLACE_COMMIT_ACTION, instantTime),
+ new HoodieInstant(false, HoodieTimeline.DELTA_COMMIT_ACTION, instantTime));
+
+ return Option.fromJavaOptional(instants.stream().filter(timeline::containsInstant).findAny());
+ }
+
+ private Option getHoodieCommitMetadata(HoodieTimeline timeline, Option hoodieInstant) throws IOException {
+ if (hoodieInstant.isPresent()) {
+ if (hoodieInstant.get().getAction().equals(HoodieTimeline.REPLACE_COMMIT_ACTION)) {
+ return Option.of(HoodieReplaceCommitMetadata.fromBytes(timeline.getInstantDetails(hoodieInstant.get()).get(),
+ HoodieReplaceCommitMetadata.class));
+ }
+ return Option.of(HoodieCommitMetadata.fromBytes(timeline.getInstantDetails(hoodieInstant.get()).get(),
+ HoodieCommitMetadata.class));
+ }
+
+ return Option.empty();
+ }
}
diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/CompactionCommand.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/CompactionCommand.java
index ffbf70e12a9bc..cb24f56236cf3 100644
--- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/CompactionCommand.java
+++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/CompactionCommand.java
@@ -22,9 +22,9 @@
import org.apache.hudi.avro.model.HoodieCompactionPlan;
import org.apache.hudi.cli.HoodieCLI;
import org.apache.hudi.cli.HoodiePrintHelper;
+import org.apache.hudi.cli.HoodieTableHeaderFields;
import org.apache.hudi.cli.TableHeader;
import org.apache.hudi.cli.commands.SparkMain.SparkCommand;
-import org.apache.hudi.cli.utils.CommitUtil;
import org.apache.hudi.cli.utils.InputStreamConsumer;
import org.apache.hudi.cli.utils.SparkUtil;
import org.apache.hudi.client.CompactionAdminClient.RenameOpResult;
@@ -48,14 +48,13 @@
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
-import org.apache.log4j.LogManager;
-import org.apache.log4j.Logger;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
import org.apache.spark.launcher.SparkLauncher;
import org.apache.spark.util.Utils;
-import org.springframework.shell.core.CommandMarker;
-import org.springframework.shell.core.annotation.CliCommand;
-import org.springframework.shell.core.annotation.CliOption;
-import org.springframework.stereotype.Component;
+import org.springframework.shell.standard.ShellComponent;
+import org.springframework.shell.standard.ShellMethod;
+import org.springframework.shell.standard.ShellOption;
import java.io.IOException;
import java.io.ObjectInputStream;
@@ -70,11 +69,13 @@
import java.util.stream.Collectors;
import java.util.stream.Stream;
+import static org.apache.hudi.cli.utils.CommitUtil.getTimeDaysAgo;
+
/**
* CLI command to display compaction related options.
*/
-@Component
-public class CompactionCommand implements CommandMarker {
+@ShellComponent
+public class CompactionCommand {
private static final Logger LOG = LogManager.getLogger(CompactionCommand.class);
@@ -88,34 +89,33 @@ private HoodieTableMetaClient checkAndGetMetaClient() {
return client;
}
- @CliCommand(value = "compactions show all", help = "Shows all compactions that are in active timeline")
+ @ShellMethod(key = "compactions show all", value = "Shows all compactions that are in active timeline")
public String compactionsAll(
- @CliOption(key = {"includeExtraMetadata"}, help = "Include extra metadata",
- unspecifiedDefaultValue = "false") final boolean includeExtraMetadata,
- @CliOption(key = {"limit"}, help = "Limit commits",
- unspecifiedDefaultValue = "-1") final Integer limit,
- @CliOption(key = {"sortBy"}, help = "Sorting Field", unspecifiedDefaultValue = "") final String sortByField,
- @CliOption(key = {"desc"}, help = "Ordering", unspecifiedDefaultValue = "false") final boolean descending,
- @CliOption(key = {"headeronly"}, help = "Print Header Only",
- unspecifiedDefaultValue = "false") final boolean headerOnly)
- throws IOException {
+ @ShellOption(value = {"--includeExtraMetadata"}, help = "Include extra metadata",
+ defaultValue = "false") final boolean includeExtraMetadata,
+ @ShellOption(value = {"--limit"}, help = "Limit commits",
+ defaultValue = "-1") final Integer limit,
+ @ShellOption(value = {"--sortBy"}, help = "Sorting Field", defaultValue = "") final String sortByField,
+ @ShellOption(value = {"--desc"}, help = "Ordering", defaultValue = "false") final boolean descending,
+ @ShellOption(value = {"--headeronly"}, help = "Print Header Only",
+ defaultValue = "false") final boolean headerOnly) {
HoodieTableMetaClient client = checkAndGetMetaClient();
HoodieActiveTimeline activeTimeline = client.getActiveTimeline();
return printAllCompactions(activeTimeline,
- compactionPlanReader(this::readCompactionPlanForActiveTimeline, activeTimeline),
- includeExtraMetadata, sortByField, descending, limit, headerOnly);
+ compactionPlanReader(this::readCompactionPlanForActiveTimeline, activeTimeline),
+ includeExtraMetadata, sortByField, descending, limit, headerOnly);
}
- @CliCommand(value = "compaction show", help = "Shows compaction details for a specific compaction instant")
+ @ShellMethod(key = "compaction show", value = "Shows compaction details for a specific compaction instant")
public String compactionShow(
- @CliOption(key = "instant", mandatory = true,
- help = "Base path for the target hoodie table") final String compactionInstantTime,
- @CliOption(key = {"limit"}, help = "Limit commits",
- unspecifiedDefaultValue = "-1") final Integer limit,
- @CliOption(key = {"sortBy"}, help = "Sorting Field", unspecifiedDefaultValue = "") final String sortByField,
- @CliOption(key = {"desc"}, help = "Ordering", unspecifiedDefaultValue = "false") final boolean descending,
- @CliOption(key = {"headeronly"}, help = "Print Header Only",
- unspecifiedDefaultValue = "false") final boolean headerOnly)
+ @ShellOption(value = "--instant",
+ help = "Base path for the target hoodie table") final String compactionInstantTime,
+ @ShellOption(value = {"--limit"}, help = "Limit commits", defaultValue = "-1") final Integer limit,
+ @ShellOption(value = {"--sortBy"}, help = "Sorting Field", defaultValue = "") final String sortByField,
+ @ShellOption(value = {"--desc"}, help = "Ordering", defaultValue = "false") final boolean descending,
+ @ShellOption(value = {"--headeronly"}, help = "Print Header Only",
+ defaultValue = "false") final boolean headerOnly,
+ @ShellOption(value = {"--partition"}, help = "Partition value", defaultValue = ShellOption.NULL) final String partition)
throws Exception {
HoodieTableMetaClient client = checkAndGetMetaClient();
HoodieActiveTimeline activeTimeline = client.getActiveTimeline();
@@ -123,77 +123,75 @@ public String compactionShow(
activeTimeline.readCompactionPlanAsBytes(
HoodieTimeline.getCompactionRequestedInstant(compactionInstantTime)).get());
- return printCompaction(compactionPlan, sortByField, descending, limit, headerOnly);
+ return printCompaction(compactionPlan, sortByField, descending, limit, headerOnly, partition);
}
- @CliCommand(value = "compactions showarchived", help = "Shows compaction details for specified time window")
+ @ShellMethod(key = "compactions showarchived", value = "Shows compaction details for specified time window")
public String compactionsShowArchived(
- @CliOption(key = {"includeExtraMetadata"}, help = "Include extra metadata",
- unspecifiedDefaultValue = "false") final boolean includeExtraMetadata,
- @CliOption(key = {"startTs"}, mandatory = false, help = "start time for compactions, default: now - 10 days")
- String startTs,
- @CliOption(key = {"endTs"}, mandatory = false, help = "end time for compactions, default: now - 1 day")
- String endTs,
- @CliOption(key = {"limit"}, help = "Limit compactions",
- unspecifiedDefaultValue = "-1") final Integer limit,
- @CliOption(key = {"sortBy"}, help = "Sorting Field", unspecifiedDefaultValue = "") final String sortByField,
- @CliOption(key = {"desc"}, help = "Ordering", unspecifiedDefaultValue = "false") final boolean descending,
- @CliOption(key = {"headeronly"}, help = "Print Header Only",
- unspecifiedDefaultValue = "false") final boolean headerOnly)
- throws Exception {
+ @ShellOption(value = {"--includeExtraMetadata"}, help = "Include extra metadata",
+ defaultValue = "false") final boolean includeExtraMetadata,
+ @ShellOption(value = {"--startTs"}, defaultValue = ShellOption.NULL,
+ help = "start time for compactions, default: now - 10 days") String startTs,
+ @ShellOption(value = {"--endTs"}, defaultValue = ShellOption.NULL,
+ help = "end time for compactions, default: now - 1 day") String endTs,
+ @ShellOption(value = {"--limit"}, help = "Limit compactions", defaultValue = "-1") final Integer limit,
+ @ShellOption(value = {"--sortBy"}, help = "Sorting Field", defaultValue = "") final String sortByField,
+ @ShellOption(value = {"--desc"}, help = "Ordering", defaultValue = "false") final boolean descending,
+ @ShellOption(value = {"--headeronly"}, help = "Print Header Only",
+ defaultValue = "false") final boolean headerOnly) {
if (StringUtils.isNullOrEmpty(startTs)) {
- startTs = CommitUtil.getTimeDaysAgo(10);
+ startTs = getTimeDaysAgo(10);
}
if (StringUtils.isNullOrEmpty(endTs)) {
- endTs = CommitUtil.getTimeDaysAgo(1);
+ endTs = getTimeDaysAgo(1);
}
HoodieTableMetaClient client = checkAndGetMetaClient();
HoodieArchivedTimeline archivedTimeline = client.getArchivedTimeline();
- archivedTimeline.loadInstantDetailsInMemory(startTs, endTs);
+ archivedTimeline.loadCompactionDetailsInMemory(startTs, endTs);
try {
return printAllCompactions(archivedTimeline,
- compactionPlanReader(this::readCompactionPlanForArchivedTimeline, archivedTimeline),
- includeExtraMetadata, sortByField, descending, limit, headerOnly);
+ compactionPlanReader(this::readCompactionPlanForArchivedTimeline, archivedTimeline),
+ includeExtraMetadata, sortByField, descending, limit, headerOnly);
} finally {
archivedTimeline.clearInstantDetailsFromMemory(startTs, endTs);
}
}
- @CliCommand(value = "compaction showarchived", help = "Shows compaction details for a specific compaction instant")
+ @ShellMethod(key = "compaction showarchived", value = "Shows compaction details for a specific compaction instant")
public String compactionShowArchived(
- @CliOption(key = "instant", mandatory = true,
- help = "instant time") final String compactionInstantTime,
- @CliOption(key = {"limit"}, help = "Limit commits",
- unspecifiedDefaultValue = "-1") final Integer limit,
- @CliOption(key = {"sortBy"}, help = "Sorting Field", unspecifiedDefaultValue = "") final String sortByField,
- @CliOption(key = {"desc"}, help = "Ordering", unspecifiedDefaultValue = "false") final boolean descending,
- @CliOption(key = {"headeronly"}, help = "Print Header Only",
- unspecifiedDefaultValue = "false") final boolean headerOnly)
- throws Exception {
+ @ShellOption(value = "--instant", help = "instant time") final String compactionInstantTime,
+ @ShellOption(value = {"--limit"}, help = "Limit commits", defaultValue = "-1") final Integer limit,
+ @ShellOption(value = {"--sortBy"}, help = "Sorting Field", defaultValue = "") final String sortByField,
+ @ShellOption(value = {"--desc"}, help = "Ordering", defaultValue = "false") final boolean descending,
+ @ShellOption(value = {"--headeronly"}, help = "Print Header Only",
+ defaultValue = "false") final boolean headerOnly,
+ @ShellOption(value = {"--partition"}, help = "Partition value", defaultValue = ShellOption.NULL) final String partition)
+ throws Exception {
HoodieTableMetaClient client = checkAndGetMetaClient();
HoodieArchivedTimeline archivedTimeline = client.getArchivedTimeline();
HoodieInstant instant = new HoodieInstant(HoodieInstant.State.COMPLETED,
- HoodieTimeline.COMPACTION_ACTION, compactionInstantTime);
- String startTs = CommitUtil.addHours(compactionInstantTime, -1);
- String endTs = CommitUtil.addHours(compactionInstantTime, 1);
+ HoodieTimeline.COMPACTION_ACTION, compactionInstantTime);
try {
- archivedTimeline.loadInstantDetailsInMemory(startTs, endTs);
- HoodieCompactionPlan compactionPlan = TimelineMetadataUtils.deserializeCompactionPlan(
- archivedTimeline.getInstantDetails(instant).get());
- return printCompaction(compactionPlan, sortByField, descending, limit, headerOnly);
+ archivedTimeline.loadCompactionDetailsInMemory(compactionInstantTime);
+ HoodieCompactionPlan compactionPlan = TimelineMetadataUtils.deserializeAvroRecordMetadata(
+ archivedTimeline.getInstantDetails(instant).get(), HoodieCompactionPlan.getClassSchema());
+ return printCompaction(compactionPlan, sortByField, descending, limit, headerOnly, partition);
} finally {
- archivedTimeline.clearInstantDetailsFromMemory(startTs, endTs);
+ archivedTimeline.clearInstantDetailsFromMemory(compactionInstantTime);
}
}
- @CliCommand(value = "compaction schedule", help = "Schedule Compaction")
- public String scheduleCompact(@CliOption(key = "sparkMemory", unspecifiedDefaultValue = "1G",
- help = "Spark executor memory") final String sparkMemory,
- @CliOption(key = "propsFilePath", help = "path to properties file on localfs or dfs with configurations for hoodie client for compacting",
- unspecifiedDefaultValue = "") final String propsFilePath,
- @CliOption(key = "hoodieConfigs", help = "Any configuration that can be set in the properties file can be passed here in the form of an array",
- unspecifiedDefaultValue = "") final String[] configs) throws Exception {
+ @ShellMethod(key = "compaction schedule", value = "Schedule Compaction")
+ public String scheduleCompact(
+ @ShellOption(value = "--sparkMemory", defaultValue = "1G",
+ help = "Spark executor memory") final String sparkMemory,
+ @ShellOption(value = "--propsFilePath", help = "path to properties file on localfs or dfs with configurations for hoodie client for compacting",
+ defaultValue = "") final String propsFilePath,
+ @ShellOption(value = "--hoodieConfigs", help = "Any configuration that can be set in the properties file can be passed here in the form of an array",
+ defaultValue = "") final String[] configs,
+ @ShellOption(value = "--sparkMaster", defaultValue = "local", help = "Spark Master") String master)
+ throws Exception {
HoodieTableMetaClient client = checkAndGetMetaClient();
boolean initialized = HoodieCLI.initConf();
HoodieCLI.initFS(initialized);
@@ -204,8 +202,9 @@ public String scheduleCompact(@CliOption(key = "sparkMemory", unspecifiedDefault
String sparkPropertiesPath =
Utils.getDefaultPropertiesFile(scala.collection.JavaConversions.propertiesAsScalaMap(System.getProperties()));
SparkLauncher sparkLauncher = SparkUtil.initLauncher(sparkPropertiesPath);
- sparkLauncher.addAppArgs(SparkCommand.COMPACT_SCHEDULE.toString(), client.getBasePath(),
- client.getTableConfig().getTableName(), compactionInstantTime, sparkMemory, propsFilePath);
+ String cmd = SparkCommand.COMPACT_SCHEDULE.toString();
+ sparkLauncher.addAppArgs(cmd, master, sparkMemory, client.getBasePath(),
+ client.getTableConfig().getTableName(), compactionInstantTime, propsFilePath);
UtilHelpers.validateAndAddProperties(configs, sparkLauncher);
Process process = sparkLauncher.launch();
InputStreamConsumer.captureOutput(process);
@@ -216,20 +215,23 @@ public String scheduleCompact(@CliOption(key = "sparkMemory", unspecifiedDefault
return "Attempted to schedule compaction for " + compactionInstantTime;
}
- @CliCommand(value = "compaction run", help = "Run Compaction for given instant time")
+ @ShellMethod(key = "compaction run", value = "Run Compaction for given instant time")
public String compact(
- @CliOption(key = {"parallelism"}, mandatory = true,
+ @ShellOption(value = {"--parallelism"}, defaultValue = "3",
help = "Parallelism for hoodie compaction") final String parallelism,
- @CliOption(key = "schemaFilePath", mandatory = true,
- help = "Path for Avro schema file") final String schemaFilePath,
- @CliOption(key = "sparkMemory", unspecifiedDefaultValue = "4G",
+ @ShellOption(value = "--schemaFilePath",
+ help = "Path for Avro schema file", defaultValue = "") final String schemaFilePath,
+ @ShellOption(value = "--sparkMaster", defaultValue = "local",
+ help = "Spark Master") String master,
+ @ShellOption(value = "--sparkMemory", defaultValue = "4G",
help = "Spark executor memory") final String sparkMemory,
- @CliOption(key = "retry", unspecifiedDefaultValue = "1", help = "Number of retries") final String retry,
- @CliOption(key = "compactionInstant", help = "Base path for the target hoodie table") String compactionInstantTime,
- @CliOption(key = "propsFilePath", help = "path to properties file on localfs or dfs with configurations for hoodie client for compacting",
- unspecifiedDefaultValue = "") final String propsFilePath,
- @CliOption(key = "hoodieConfigs", help = "Any configuration that can be set in the properties file can be passed here in the form of an array",
- unspecifiedDefaultValue = "") final String[] configs)
+ @ShellOption(value = "--retry", defaultValue = "1", help = "Number of retries") final String retry,
+ @ShellOption(value = "--compactionInstant", help = "Instant of compaction.request",
+ defaultValue = ShellOption.NULL) String compactionInstantTime,
+ @ShellOption(value = "--propsFilePath", help = "path to properties file on localfs or dfs with configurations for hoodie client for compacting",
+ defaultValue = "") final String propsFilePath,
+ @ShellOption(value = "--hoodieConfigs", help = "Any configuration that can be set in the properties file can be passed here in the form of an array",
+ defaultValue = "") final String[] configs)
throws Exception {
HoodieTableMetaClient client = checkAndGetMetaClient();
boolean initialized = HoodieCLI.initConf();
@@ -249,9 +251,9 @@ public String compact(
String sparkPropertiesPath =
Utils.getDefaultPropertiesFile(scala.collection.JavaConversions.propertiesAsScalaMap(System.getProperties()));
SparkLauncher sparkLauncher = SparkUtil.initLauncher(sparkPropertiesPath);
- sparkLauncher.addAppArgs(SparkCommand.COMPACT_RUN.toString(), client.getBasePath(),
+ sparkLauncher.addAppArgs(SparkCommand.COMPACT_RUN.toString(), master, sparkMemory, client.getBasePath(),
client.getTableConfig().getTableName(), compactionInstantTime, parallelism, schemaFilePath,
- sparkMemory, retry, propsFilePath);
+ retry, propsFilePath);
UtilHelpers.validateAndAddProperties(configs, sparkLauncher);
Process process = sparkLauncher.launch();
InputStreamConsumer.captureOutput(process);
@@ -262,32 +264,67 @@ public String compact(
return "Compaction successfully completed for " + compactionInstantTime;
}
+ @ShellMethod(key = "compaction scheduleAndExecute", value = "Schedule compaction plan and execute this plan")
+ public String compact(
+ @ShellOption(value = {"--parallelism"}, defaultValue = "3",
+ help = "Parallelism for hoodie compaction") final String parallelism,
+ @ShellOption(value = "--schemaFilePath",
+ help = "Path for Avro schema file", defaultValue = ShellOption.NULL) final String schemaFilePath,
+ @ShellOption(value = "--sparkMaster", defaultValue = "local",
+ help = "Spark Master") String master,
+ @ShellOption(value = "--sparkMemory", defaultValue = "4G",
+ help = "Spark executor memory") final String sparkMemory,
+ @ShellOption(value = "--retry", defaultValue = "1", help = "Number of retries") final String retry,
+ @ShellOption(value = "--propsFilePath", help = "path to properties file on localfs or dfs with configurations for hoodie client for compacting",
+ defaultValue = "") final String propsFilePath,
+ @ShellOption(value = "--hoodieConfigs", help = "Any configuration that can be set in the properties file can be passed here in the form of an array",
+ defaultValue = "") final String[] configs)
+ throws Exception {
+ HoodieTableMetaClient client = checkAndGetMetaClient();
+ boolean initialized = HoodieCLI.initConf();
+ HoodieCLI.initFS(initialized);
+ String sparkPropertiesPath =
+ Utils.getDefaultPropertiesFile(scala.collection.JavaConversions.propertiesAsScalaMap(System.getProperties()));
+ SparkLauncher sparkLauncher = SparkUtil.initLauncher(sparkPropertiesPath);
+ sparkLauncher.addAppArgs(SparkCommand.COMPACT_SCHEDULE_AND_EXECUTE.toString(), master, sparkMemory, client.getBasePath(),
+ client.getTableConfig().getTableName(), parallelism, schemaFilePath,
+ retry, propsFilePath);
+ UtilHelpers.validateAndAddProperties(configs, sparkLauncher);
+ Process process = sparkLauncher.launch();
+ InputStreamConsumer.captureOutput(process);
+ int exitCode = process.waitFor();
+ if (exitCode != 0) {
+ return "Failed to schedule and execute compaction ";
+ }
+ return "Schedule and execute compaction successfully completed";
+ }
+
/**
* Prints all compaction details.
*/
- private String printAllCompactions(HoodieDefaultTimeline timeline,
- Function compactionPlanReader,
- boolean includeExtraMetadata,
- String sortByField,
- boolean descending,
- int limit,
- boolean headerOnly) {
-
- Stream instantsStream = timeline.getCommitsAndCompactionTimeline().getReverseOrderedInstants();
+ private static String printAllCompactions(HoodieDefaultTimeline timeline,
+ Function compactionPlanReader,
+ boolean includeExtraMetadata,
+ String sortByField,
+ boolean descending,
+ int limit,
+ boolean headerOnly) {
+
+ Stream instantsStream = timeline.getWriteTimeline().getReverseOrderedInstants();
List> compactionPlans = instantsStream
- .map(instant -> Pair.of(instant, compactionPlanReader.apply(instant)))
- .filter(pair -> pair.getRight() != null)
- .collect(Collectors.toList());
+ .map(instant -> Pair.of(instant, compactionPlanReader.apply(instant)))
+ .filter(pair -> pair.getRight() != null)
+ .collect(Collectors.toList());
- Set committedInstants = timeline.getCommitTimeline().filterCompletedInstants()
- .getInstants().collect(Collectors.toSet());
+ Set committedInstants = timeline.getCommitTimeline().filterCompletedInstants()
+ .getInstants().map(HoodieInstant::getTimestamp).collect(Collectors.toSet());
List rows = new ArrayList<>();
for (Pair compactionPlan : compactionPlans) {
HoodieCompactionPlan plan = compactionPlan.getRight();
HoodieInstant instant = compactionPlan.getLeft();
final HoodieInstant.State state;
- if (committedInstants.contains(instant)) {
+ if (committedInstants.contains(instant.getTimestamp())) {
state = HoodieInstant.State.COMPLETED;
} else {
state = instant.getState();
@@ -295,19 +332,21 @@ private String printAllCompactions(HoodieDefaultTimeline timeline,
if (includeExtraMetadata) {
rows.add(new Comparable[] {instant.getTimestamp(), state.toString(),
- plan.getOperations() == null ? 0 : plan.getOperations().size(),
- plan.getExtraMetadata().toString()});
+ plan.getOperations() == null ? 0 : plan.getOperations().size(),
+ plan.getExtraMetadata().toString()});
} else {
rows.add(new Comparable[] {instant.getTimestamp(), state.toString(),
- plan.getOperations() == null ? 0 : plan.getOperations().size()});
+ plan.getOperations() == null ? 0 : plan.getOperations().size()});
}
}
Map> fieldNameToConverterMap = new HashMap<>();
- TableHeader header = new TableHeader().addTableHeaderField("Compaction Instant Time").addTableHeaderField("State")
- .addTableHeaderField("Total FileIds to be Compacted");
+ TableHeader header = new TableHeader()
+ .addTableHeaderField(HoodieTableHeaderFields.HEADER_COMPACTION_INSTANT_TIME)
+ .addTableHeaderField(HoodieTableHeaderFields.HEADER_STATE)
+ .addTableHeaderField(HoodieTableHeaderFields.HEADER_TOTAL_FILES_TO_BE_COMPACTED);
if (includeExtraMetadata) {
- header = header.addTableHeaderField("Extra Metadata");
+ header = header.addTableHeaderField(HoodieTableHeaderFields.HEADER_EXTRA_METADATA);
}
return HoodiePrintHelper.print(header, fieldNameToConverterMap, sortByField, descending, limit, headerOnly, rows);
}
@@ -319,21 +358,24 @@ private String printAllCompactions(HoodieDefaultTimeline timeline,
*/
private
Function compactionPlanReader(
- BiFunction f, T timeline) {
+ BiFunction f, T timeline) {
return (y) -> f.apply(timeline, y);
}
private HoodieCompactionPlan readCompactionPlanForArchivedTimeline(HoodieArchivedTimeline archivedTimeline,
HoodieInstant instant) {
- if (!HoodieTimeline.COMPACTION_ACTION.equals(instant.getAction())) {
- return null;
- } else {
+ // filter inflight compaction
+ if (HoodieTimeline.COMPACTION_ACTION.equals(instant.getAction())
+ && HoodieInstant.State.INFLIGHT.equals(instant.getState())) {
try {
- return TimelineMetadataUtils.deserializeCompactionPlan(archivedTimeline.getInstantDetails(instant).get());
- } catch (IOException e) {
- throw new HoodieIOException(e.getMessage(), e);
+ return TimelineMetadataUtils.deserializeAvroRecordMetadata(archivedTimeline.getInstantDetails(instant).get(),
+ HoodieCompactionPlan.getClassSchema());
+ } catch (Exception e) {
+ throw new HoodieException(e.getMessage(), e);
}
+ } else {
+ return null;
}
}
@@ -347,38 +389,45 @@ private HoodieCompactionPlan readCompactionPlanForActiveTimeline(HoodieActiveTim
try {
// This could be a completed compaction. Assume a compaction request file is present but skip if fails
return TimelineMetadataUtils.deserializeCompactionPlan(
- activeTimeline.readCompactionPlanAsBytes(
- HoodieTimeline.getCompactionRequestedInstant(instant.getTimestamp())).get());
+ activeTimeline.readCompactionPlanAsBytes(
+ HoodieTimeline.getCompactionRequestedInstant(instant.getTimestamp())).get());
} catch (HoodieIOException ioe) {
// SKIP
return null;
}
} else {
return TimelineMetadataUtils.deserializeCompactionPlan(activeTimeline.readCompactionPlanAsBytes(
- HoodieTimeline.getCompactionRequestedInstant(instant.getTimestamp())).get());
+ HoodieTimeline.getCompactionRequestedInstant(instant.getTimestamp())).get());
}
} catch (IOException e) {
throw new HoodieIOException(e.getMessage(), e);
}
}
- private String printCompaction(HoodieCompactionPlan compactionPlan,
- String sortByField,
- boolean descending,
- int limit,
- boolean headerOnly) {
+ protected static String printCompaction(HoodieCompactionPlan compactionPlan,
+ String sortByField,
+ boolean descending,
+ int limit,
+ boolean headerOnly,
+ final String partition) {
List rows = new ArrayList<>();
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()});
+ if (StringUtils.isNullOrEmpty(partition) || partition.equals(op.getPartitionPath())) {
+ rows.add(new Comparable[] {op.getPartitionPath(), op.getFileId(), op.getBaseInstantTime(), op.getDataFilePath(),
+ op.getDeltaFilePaths().size(), op.getMetrics() == null ? "" : op.getMetrics().toString()});
+ }
}
}
Map> fieldNameToConverterMap = new HashMap<>();
- TableHeader header = new TableHeader().addTableHeaderField("Partition Path").addTableHeaderField("File Id")
- .addTableHeaderField("Base Instant").addTableHeaderField("Data File Path")
- .addTableHeaderField("Total Delta Files").addTableHeaderField("getMetrics");
+ TableHeader header = new TableHeader()
+ .addTableHeaderField(HoodieTableHeaderFields.HEADER_PARTITION_PATH)
+ .addTableHeaderField(HoodieTableHeaderFields.HEADER_FILE_ID)
+ .addTableHeaderField(HoodieTableHeaderFields.HEADER_BASE_INSTANT)
+ .addTableHeaderField(HoodieTableHeaderFields.HEADER_DATA_FILE_PATH)
+ .addTableHeaderField(HoodieTableHeaderFields.HEADER_TOTAL_DELTA_FILES)
+ .addTableHeaderField(HoodieTableHeaderFields.HEADER_METRICS);
return HoodiePrintHelper.print(header, fieldNameToConverterMap, sortByField, descending, limit, headerOnly, rows);
}
@@ -400,17 +449,17 @@ private T deSerializeOperationResult(String inputP, FileSystem fs) throws Ex
}
}
- @CliCommand(value = "compaction validate", help = "Validate Compaction")
+ @ShellMethod(key = "compaction validate", value = "Validate Compaction")
public String validateCompaction(
- @CliOption(key = "instant", mandatory = true, help = "Compaction Instant") String compactionInstant,
- @CliOption(key = {"parallelism"}, unspecifiedDefaultValue = "3", help = "Parallelism") String parallelism,
- @CliOption(key = "sparkMaster", unspecifiedDefaultValue = "", help = "Spark Master ") String master,
- @CliOption(key = "sparkMemory", unspecifiedDefaultValue = "2G", help = "executor memory") String sparkMemory,
- @CliOption(key = {"limit"}, help = "Limit commits", unspecifiedDefaultValue = "-1") Integer limit,
- @CliOption(key = {"sortBy"}, help = "Sorting Field", unspecifiedDefaultValue = "") String sortByField,
- @CliOption(key = {"desc"}, help = "Ordering", unspecifiedDefaultValue = "false") boolean descending,
- @CliOption(key = {"headeronly"}, help = "Print Header Only",
- unspecifiedDefaultValue = "false") boolean headerOnly)
+ @ShellOption(value = "--instant", help = "Compaction Instant") String compactionInstant,
+ @ShellOption(value = {"--parallelism"}, defaultValue = "3", help = "Parallelism") String parallelism,
+ @ShellOption(value = "--sparkMaster", defaultValue = "local", help = "Spark Master") String master,
+ @ShellOption(value = "--sparkMemory", defaultValue = "2G", help = "executor memory") String sparkMemory,
+ @ShellOption(value = {"--limit"}, help = "Limit commits", defaultValue = "-1") Integer limit,
+ @ShellOption(value = {"--sortBy"}, help = "Sorting Field", defaultValue = "") String sortByField,
+ @ShellOption(value = {"--desc"}, help = "Ordering", defaultValue = "false") boolean descending,
+ @ShellOption(value = {"--headeronly"}, help = "Print Header Only",
+ defaultValue = "false") boolean headerOnly)
throws Exception {
HoodieTableMetaClient client = checkAndGetMetaClient();
boolean initialized = HoodieCLI.initConf();
@@ -444,9 +493,13 @@ public String validateCompaction(
});
Map> fieldNameToConverterMap = new HashMap<>();
- TableHeader header = new TableHeader().addTableHeaderField("File Id").addTableHeaderField("Base Instant Time")
- .addTableHeaderField("Base Data File").addTableHeaderField("Num Delta Files").addTableHeaderField("Valid")
- .addTableHeaderField("Error");
+ TableHeader header = new TableHeader()
+ .addTableHeaderField(HoodieTableHeaderFields.HEADER_FILE_ID)
+ .addTableHeaderField(HoodieTableHeaderFields.HEADER_BASE_INSTANT_TIME)
+ .addTableHeaderField(HoodieTableHeaderFields.HEADER_BASE_DATA_FILE)
+ .addTableHeaderField(HoodieTableHeaderFields.HEADER_NUM_DELTA_FILES)
+ .addTableHeaderField(HoodieTableHeaderFields.HEADER_VALID)
+ .addTableHeaderField(HoodieTableHeaderFields.HEADER_ERROR);
output = message + HoodiePrintHelper.print(header, fieldNameToConverterMap, sortByField, descending, limit,
headerOnly, rows);
@@ -459,19 +512,19 @@ public String validateCompaction(
return output;
}
- @CliCommand(value = "compaction unschedule", help = "Unschedule Compaction")
+ @ShellMethod(key = "compaction unschedule", value = "Unschedule Compaction")
public String unscheduleCompaction(
- @CliOption(key = "instant", mandatory = true, help = "Compaction Instant") String compactionInstant,
- @CliOption(key = {"parallelism"}, unspecifiedDefaultValue = "3", help = "Parallelism") String parallelism,
- @CliOption(key = "sparkMaster", unspecifiedDefaultValue = "", help = "Spark Master ") String master,
- @CliOption(key = "sparkMemory", unspecifiedDefaultValue = "2G", help = "executor memory") String sparkMemory,
- @CliOption(key = {"skipValidation"}, help = "skip validation", unspecifiedDefaultValue = "false") boolean skipV,
- @CliOption(key = {"dryRun"}, help = "Dry Run Mode", unspecifiedDefaultValue = "false") boolean dryRun,
- @CliOption(key = {"limit"}, help = "Limit commits", unspecifiedDefaultValue = "-1") Integer limit,
- @CliOption(key = {"sortBy"}, help = "Sorting Field", unspecifiedDefaultValue = "") String sortByField,
- @CliOption(key = {"desc"}, help = "Ordering", unspecifiedDefaultValue = "false") boolean descending,
- @CliOption(key = {"headeronly"}, help = "Print Header Only",
- unspecifiedDefaultValue = "false") boolean headerOnly)
+ @ShellOption(value = "--instant", help = "Compaction Instant") String compactionInstant,
+ @ShellOption(value = {"--parallelism"}, defaultValue = "3", help = "Parallelism") String parallelism,
+ @ShellOption(value = "--sparkMaster", defaultValue = "local", help = "Spark Master") String master,
+ @ShellOption(value = "--sparkMemory", defaultValue = "2G", help = "executor memory") String sparkMemory,
+ @ShellOption(value = {"--skipValidation"}, help = "skip validation", defaultValue = "false") boolean skipV,
+ @ShellOption(value = {"--dryRun"}, help = "Dry Run Mode", defaultValue = "false") boolean dryRun,
+ @ShellOption(value = {"--limit"}, help = "Limit commits", defaultValue = "-1") Integer limit,
+ @ShellOption(value = {"--sortBy"}, help = "Sorting Field", defaultValue = "") String sortByField,
+ @ShellOption(value = {"--desc"}, help = "Ordering", defaultValue = "false") boolean descending,
+ @ShellOption(value = {"--headeronly"}, help = "Print Header Only",
+ defaultValue = "false") boolean headerOnly)
throws Exception {
HoodieTableMetaClient client = checkAndGetMetaClient();
boolean initialized = HoodieCLI.initConf();
@@ -505,17 +558,18 @@ public String unscheduleCompaction(
return output;
}
- @CliCommand(value = "compaction unscheduleFileId", help = "UnSchedule Compaction for a fileId")
+ @ShellMethod(key = "compaction unscheduleFileId", value = "UnSchedule Compaction for a fileId")
public String unscheduleCompactFile(
- @CliOption(key = "fileId", mandatory = true, help = "File Id") final String fileId,
- @CliOption(key = "sparkMaster", unspecifiedDefaultValue = "", help = "Spark Master ") String master,
- @CliOption(key = "sparkMemory", unspecifiedDefaultValue = "2G", help = "executor memory") String sparkMemory,
- @CliOption(key = {"skipValidation"}, help = "skip validation", unspecifiedDefaultValue = "false") boolean skipV,
- @CliOption(key = {"dryRun"}, help = "Dry Run Mode", unspecifiedDefaultValue = "false") boolean dryRun,
- @CliOption(key = {"limit"}, help = "Limit commits", unspecifiedDefaultValue = "-1") Integer limit,
- @CliOption(key = {"sortBy"}, help = "Sorting Field", unspecifiedDefaultValue = "") String sortByField,
- @CliOption(key = {"desc"}, help = "Ordering", unspecifiedDefaultValue = "false") boolean descending,
- @CliOption(key = {"headeronly"}, help = "Header Only", unspecifiedDefaultValue = "false") boolean headerOnly)
+ @ShellOption(value = "--fileId", help = "File Id") final String fileId,
+ @ShellOption(value = "--partitionPath", defaultValue = "", help = "partition path") final String partitionPath,
+ @ShellOption(value = "--sparkMaster", defaultValue = "local", help = "Spark Master") String master,
+ @ShellOption(value = "--sparkMemory", defaultValue = "2G", help = "executor memory") String sparkMemory,
+ @ShellOption(value = {"--skipValidation"}, help = "skip validation", defaultValue = "false") boolean skipV,
+ @ShellOption(value = {"--dryRun"}, help = "Dry Run Mode", defaultValue = "false") boolean dryRun,
+ @ShellOption(value = {"--limit"}, help = "Limit commits", defaultValue = "-1") Integer limit,
+ @ShellOption(value = {"--sortBy"}, help = "Sorting Field", defaultValue = "") String sortByField,
+ @ShellOption(value = {"--desc"}, help = "Ordering", defaultValue = "false") boolean descending,
+ @ShellOption(value = {"--headeronly"}, help = "Header Only", defaultValue = "false") boolean headerOnly)
throws Exception {
HoodieTableMetaClient client = checkAndGetMetaClient();
boolean initialized = HoodieCLI.initConf();
@@ -529,7 +583,7 @@ public String unscheduleCompactFile(
.getDefaultPropertiesFile(scala.collection.JavaConversions.propertiesAsScalaMap(System.getProperties()));
SparkLauncher sparkLauncher = SparkUtil.initLauncher(sparkPropertiesPath);
sparkLauncher.addAppArgs(SparkCommand.COMPACT_UNSCHEDULE_FILE.toString(), master, sparkMemory, client.getBasePath(),
- fileId, outputPathStr, "1", Boolean.valueOf(skipV).toString(),
+ fileId, partitionPath, outputPathStr, "1", Boolean.valueOf(skipV).toString(),
Boolean.valueOf(dryRun).toString());
Process process = sparkLauncher.launch();
InputStreamConsumer.captureOutput(process);
@@ -549,19 +603,19 @@ public String unscheduleCompactFile(
return output;
}
- @CliCommand(value = "compaction repair", help = "Renames the files to make them consistent with the timeline as "
+ @ShellMethod(key = "compaction repair", value = "Renames the files to make them consistent with the timeline as "
+ "dictated by Hoodie metadata. Use when compaction unschedule fails partially.")
public String repairCompaction(
- @CliOption(key = "instant", mandatory = true, help = "Compaction Instant") String compactionInstant,
- @CliOption(key = {"parallelism"}, unspecifiedDefaultValue = "3", help = "Parallelism") String parallelism,
- @CliOption(key = "sparkMaster", unspecifiedDefaultValue = "", help = "Spark Master ") String master,
- @CliOption(key = "sparkMemory", unspecifiedDefaultValue = "2G", help = "executor memory") String sparkMemory,
- @CliOption(key = {"dryRun"}, help = "Dry Run Mode", unspecifiedDefaultValue = "false") boolean dryRun,
- @CliOption(key = {"limit"}, help = "Limit commits", unspecifiedDefaultValue = "-1") Integer limit,
- @CliOption(key = {"sortBy"}, help = "Sorting Field", unspecifiedDefaultValue = "") String sortByField,
- @CliOption(key = {"desc"}, help = "Ordering", unspecifiedDefaultValue = "false") boolean descending,
- @CliOption(key = {"headeronly"}, help = "Print Header Only",
- unspecifiedDefaultValue = "false") boolean headerOnly)
+ @ShellOption(value = "--instant", help = "Compaction Instant") String compactionInstant,
+ @ShellOption(value = {"--parallelism"}, defaultValue = "3", help = "Parallelism") String parallelism,
+ @ShellOption(value = "--sparkMaster", defaultValue = "local", help = "Spark Master") String master,
+ @ShellOption(value = "--sparkMemory", defaultValue = "2G", help = "executor memory") String sparkMemory,
+ @ShellOption(value = {"--dryRun"}, help = "Dry Run Mode", defaultValue = "false") boolean dryRun,
+ @ShellOption(value = {"--limit"}, help = "Limit commits", defaultValue = "-1") Integer limit,
+ @ShellOption(value = {"--sortBy"}, help = "Sorting Field", defaultValue = "") String sortByField,
+ @ShellOption(value = {"--desc"}, help = "Ordering", defaultValue = "false") boolean descending,
+ @ShellOption(value = {"--headeronly"}, help = "Print Header Only",
+ defaultValue = "false") boolean headerOnly)
throws Exception {
HoodieTableMetaClient client = checkAndGetMetaClient();
boolean initialized = HoodieCLI.initConf();
@@ -594,7 +648,7 @@ public String repairCompaction(
}
private String getRenamesToBePrinted(List res, Integer limit, String sortByField, boolean descending,
- boolean headerOnly, String operation) {
+ boolean headerOnly, String operation) {
Option result =
Option.fromJavaOptional(res.stream().map(r -> r.isExecuted() && r.isSuccess()).reduce(Boolean::logicalAnd));
@@ -616,9 +670,13 @@ private String getRenamesToBePrinted(List res, Integer limit, St
});
Map> fieldNameToConverterMap = new HashMap<>();
- TableHeader header = new TableHeader().addTableHeaderField("File Id").addTableHeaderField("Source File Path")
- .addTableHeaderField("Destination File Path").addTableHeaderField("Rename Executed?")
- .addTableHeaderField("Rename Succeeded?").addTableHeaderField("Error");
+ TableHeader header = new TableHeader()
+ .addTableHeaderField(HoodieTableHeaderFields.HEADER_FILE_ID)
+ .addTableHeaderField(HoodieTableHeaderFields.HEADER_SOURCE_FILE_PATH)
+ .addTableHeaderField(HoodieTableHeaderFields.HEADER_DESTINATION_FILE_PATH)
+ .addTableHeaderField(HoodieTableHeaderFields.HEADER_RENAME_EXECUTED)
+ .addTableHeaderField(HoodieTableHeaderFields.HEADER_RENAME_SUCCEEDED)
+ .addTableHeaderField(HoodieTableHeaderFields.HEADER_ERROR);
return HoodiePrintHelper.print(header, fieldNameToConverterMap, sortByField, descending, limit, headerOnly, rows);
} else {
diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/DiffCommand.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/DiffCommand.java
new file mode 100644
index 0000000000000..07d21fe022668
--- /dev/null
+++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/DiffCommand.java
@@ -0,0 +1,187 @@
+/*
+ * 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.cli.commands;
+
+import org.apache.hudi.cli.HoodieCLI;
+import org.apache.hudi.cli.HoodiePrintHelper;
+import org.apache.hudi.cli.HoodieTableHeaderFields;
+import org.apache.hudi.common.model.HoodieCommitMetadata;
+import org.apache.hudi.common.model.HoodieWriteStat;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
+import org.apache.hudi.common.table.timeline.HoodieArchivedTimeline;
+import org.apache.hudi.common.table.timeline.HoodieDefaultTimeline;
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.common.util.NumericUtils;
+import org.apache.hudi.common.util.Option;
+
+import org.springframework.shell.standard.ShellComponent;
+import org.springframework.shell.standard.ShellMethod;
+import org.springframework.shell.standard.ShellOption;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.function.BiFunction;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+import static org.apache.hudi.cli.utils.CommitUtil.getTimeDaysAgo;
+import static org.apache.hudi.common.util.StringUtils.isNullOrEmpty;
+import static org.apache.hudi.common.util.StringUtils.nonEmpty;
+import static org.apache.hudi.common.util.ValidationUtils.checkArgument;
+
+/**
+ * Given a file id or partition value, this command line utility tracks the changes to the file group or partition across range of commits.
+ * Usage: diff file --fileId
+ */
+@ShellComponent
+public class DiffCommand {
+
+ private static final BiFunction FILE_ID_CHECKER = (writeStat, fileId) -> fileId.equals(writeStat.getFileId());
+ private static final BiFunction PARTITION_CHECKER = (writeStat, partitionPath) -> partitionPath.equals(writeStat.getPartitionPath());
+
+ @ShellMethod(key = "diff file", value = "Check how file differs across range of commits")
+ public String diffFile(
+ @ShellOption(value = {"--fileId"}, help = "File ID to diff across range of commits") String fileId,
+ @ShellOption(value = {"--startTs"}, help = "start time for compactions, default: now - 10 days",
+ defaultValue = ShellOption.NULL) String startTs,
+ @ShellOption(value = {"--endTs"}, help = "end time for compactions, default: now - 1 day",
+ defaultValue = ShellOption.NULL) String endTs,
+ @ShellOption(value = {"--limit"}, help = "Limit compactions", defaultValue = "-1") final Integer limit,
+ @ShellOption(value = {"--sortBy"}, help = "Sorting Field", defaultValue = "") final String sortByField,
+ @ShellOption(value = {"--desc"}, help = "Ordering", defaultValue = "false") final boolean descending,
+ @ShellOption(value = {"--headeronly"}, help = "Print Header Only", defaultValue = "false") final boolean headerOnly,
+ @ShellOption(value = {"--includeArchivedTimeline"}, help = "Include archived commits as well",
+ defaultValue = "false") final boolean includeArchivedTimeline) throws IOException {
+ HoodieDefaultTimeline timeline = getTimelineInRange(startTs, endTs, includeArchivedTimeline);
+ return printCommitsWithMetadataForFileId(timeline, limit, sortByField, descending, headerOnly, "", fileId);
+ }
+
+ @ShellMethod(key = "diff partition", value = "Check how file differs across range of commits. It is meant to be used only for partitioned tables.")
+ public String diffPartition(
+ @ShellOption(value = {"--partitionPath"}, help = "Relative partition path to diff across range of commits") String partitionPath,
+ @ShellOption(value = {"--startTs"}, help = "start time for compactions, default: now - 10 days",
+ defaultValue = ShellOption.NULL) String startTs,
+ @ShellOption(value = {"--endTs"}, help = "end time for compactions, default: now - 1 day",
+ defaultValue = ShellOption.NULL) String endTs,
+ @ShellOption(value = {"--limit"}, help = "Limit compactions", defaultValue = "-1") final Integer limit,
+ @ShellOption(value = {"--sortBy"}, help = "Sorting Field", defaultValue = "") final String sortByField,
+ @ShellOption(value = {"--desc"}, help = "Ordering", defaultValue = "false") final boolean descending,
+ @ShellOption(value = {"--headeronly"}, help = "Print Header Only", defaultValue = "false") final boolean headerOnly,
+ @ShellOption(value = {"--includeArchivedTimeline"}, help = "Include archived commits as well",
+ defaultValue = "false") final boolean includeArchivedTimeline) throws IOException {
+ HoodieDefaultTimeline timeline = getTimelineInRange(startTs, endTs, includeArchivedTimeline);
+ return printCommitsWithMetadataForPartition(timeline, limit, sortByField, descending, headerOnly, "", partitionPath);
+ }
+
+ private HoodieDefaultTimeline getTimelineInRange(String startTs, String endTs, boolean includeArchivedTimeline) {
+ if (isNullOrEmpty(startTs)) {
+ startTs = getTimeDaysAgo(10);
+ }
+ if (isNullOrEmpty(endTs)) {
+ endTs = getTimeDaysAgo(1);
+ }
+ checkArgument(nonEmpty(startTs), "startTs is null or empty");
+ checkArgument(nonEmpty(endTs), "endTs is null or empty");
+ HoodieTableMetaClient metaClient = HoodieCLI.getTableMetaClient();
+ HoodieActiveTimeline activeTimeline = metaClient.getActiveTimeline();
+ if (includeArchivedTimeline) {
+ HoodieArchivedTimeline archivedTimeline = metaClient.getArchivedTimeline();
+ archivedTimeline.loadInstantDetailsInMemory(startTs, endTs);
+ return archivedTimeline.findInstantsInRange(startTs, endTs).mergeTimeline(activeTimeline);
+ }
+ return activeTimeline;
+ }
+
+ private String printCommitsWithMetadataForFileId(HoodieDefaultTimeline timeline,
+ final Integer limit,
+ final String sortByField,
+ final boolean descending,
+ final boolean headerOnly,
+ final String tempTableName,
+ final String fileId) throws IOException {
+ return printDiffWithMetadata(timeline, limit, sortByField, descending, headerOnly, tempTableName, fileId, FILE_ID_CHECKER);
+ }
+
+ private String printCommitsWithMetadataForPartition(HoodieDefaultTimeline timeline,
+ final Integer limit,
+ final String sortByField,
+ final boolean descending,
+ final boolean headerOnly,
+ final String tempTableName,
+ final String partition) throws IOException {
+ return printDiffWithMetadata(timeline, limit, sortByField, descending, headerOnly, tempTableName, partition, PARTITION_CHECKER);
+ }
+
+ private String printDiffWithMetadata(HoodieDefaultTimeline timeline, Integer limit, String sortByField, boolean descending, boolean headerOnly, String tempTableName, String diffEntity,
+ BiFunction diffEntityChecker) throws IOException {
+ List rows = new ArrayList<>();
+ List commits = timeline.getCommitsTimeline().filterCompletedInstants()
+ .getInstants().sorted(HoodieInstant.COMPARATOR.reversed()).collect(Collectors.toList());
+
+ for (final HoodieInstant commit : commits) {
+ Option instantDetails = timeline.getInstantDetails(commit);
+ if (instantDetails.isPresent()) {
+ HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes(instantDetails.get(), HoodieCommitMetadata.class);
+ for (Map.Entry> partitionWriteStat :
+ commitMetadata.getPartitionToWriteStats().entrySet()) {
+ for (HoodieWriteStat hoodieWriteStat : partitionWriteStat.getValue()) {
+ populateRows(rows, commit, hoodieWriteStat, diffEntity, diffEntityChecker);
+ }
+ }
+ }
+ }
+
+ Map> fieldNameToConverterMap = new HashMap<>();
+ fieldNameToConverterMap.put(
+ HoodieTableHeaderFields.HEADER_TOTAL_BYTES_WRITTEN,
+ entry -> NumericUtils.humanReadableByteCount((Double.parseDouble(entry.toString()))));
+
+ return HoodiePrintHelper.print(HoodieTableHeaderFields.getTableHeaderWithExtraMetadata(),
+ fieldNameToConverterMap, sortByField, descending, limit, headerOnly, rows, tempTableName);
+ }
+
+ private void populateRows(List rows, HoodieInstant commit, HoodieWriteStat hoodieWriteStat,
+ String value, BiFunction checker) {
+ if (checker.apply(hoodieWriteStat, value)) {
+ rows.add(new Comparable[] {
+ commit.getAction(),
+ commit.getTimestamp(),
+ hoodieWriteStat.getPartitionPath(),
+ hoodieWriteStat.getFileId(),
+ hoodieWriteStat.getPrevCommit(),
+ hoodieWriteStat.getNumWrites(),
+ hoodieWriteStat.getNumInserts(),
+ hoodieWriteStat.getNumDeletes(),
+ hoodieWriteStat.getNumUpdateWrites(),
+ hoodieWriteStat.getTotalWriteErrors(),
+ hoodieWriteStat.getTotalLogBlocks(),
+ hoodieWriteStat.getTotalCorruptLogBlock(),
+ hoodieWriteStat.getTotalRollbackBlocks(),
+ hoodieWriteStat.getTotalLogRecords(),
+ hoodieWriteStat.getTotalUpdatedRecordsCompacted(),
+ hoodieWriteStat.getTotalWriteBytes()
+ });
+ }
+ }
+}
diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/ExportCommand.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/ExportCommand.java
index 8bd842c825659..2406eddacf320 100644
--- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/ExportCommand.java
+++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/ExportCommand.java
@@ -18,6 +18,12 @@
package org.apache.hudi.cli.commands;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.avro.generic.IndexedRecord;
+import org.apache.avro.specific.SpecificData;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
import org.apache.hudi.avro.HoodieAvroUtils;
import org.apache.hudi.avro.model.HoodieArchivedMetaEntry;
import org.apache.hudi.avro.model.HoodieCleanMetadata;
@@ -34,18 +40,11 @@
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.common.table.timeline.TimelineMetadataUtils;
+import org.apache.hudi.common.util.ClosableIterator;
import org.apache.hudi.exception.HoodieException;
-import org.apache.log4j.LogManager;
-import org.apache.log4j.Logger;
-import org.apache.avro.generic.GenericRecord;
-import org.apache.avro.generic.IndexedRecord;
-import org.apache.avro.specific.SpecificData;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.Path;
-import org.springframework.shell.core.CommandMarker;
-import org.springframework.shell.core.annotation.CliCommand;
-import org.springframework.shell.core.annotation.CliOption;
-import org.springframework.stereotype.Component;
+import org.springframework.shell.standard.ShellComponent;
+import org.springframework.shell.standard.ShellMethod;
+import org.springframework.shell.standard.ShellOption;
import java.io.File;
import java.io.FileOutputStream;
@@ -58,21 +57,21 @@
/**
* CLI commands to export various information from a HUDI dataset.
- *
+ *
* "export instants": Export Instants and their metadata from the Timeline to a local
- * directory specified by the parameter --localFolder
- * The instants are exported in the json format.
+ * directory specified by the parameter --localFolder
+ * The instants are exported in the json format.
*/
-@Component
-public class ExportCommand implements CommandMarker {
+@ShellComponent
+public class ExportCommand {
- @CliCommand(value = "export instants", help = "Export Instants and their metadata from the Timeline")
+ @ShellMethod(key = "export instants", value = "Export Instants and their metadata from the Timeline")
public String exportInstants(
- @CliOption(key = {"limit"}, help = "Limit Instants", unspecifiedDefaultValue = "-1") final Integer limit,
- @CliOption(key = {"actions"}, help = "Comma seperated list of Instant actions to export",
- unspecifiedDefaultValue = "clean,commit,deltacommit,rollback,savepoint,restore") final String filter,
- @CliOption(key = {"desc"}, help = "Ordering", unspecifiedDefaultValue = "false") final boolean descending,
- @CliOption(key = {"localFolder"}, help = "Local Folder to export to", mandatory = true) String localFolder)
+ @ShellOption(value = {"--limit"}, help = "Limit Instants", defaultValue = "-1") final Integer limit,
+ @ShellOption(value = {"--actions"}, help = "Comma separated list of Instant actions to export",
+ defaultValue = "clean,commit,deltacommit,rollback,savepoint,restore") final String filter,
+ @ShellOption(value = {"--desc"}, help = "Ordering", defaultValue = "false") final boolean descending,
+ @ShellOption(value = {"--localFolder"}, help = "Local Folder to export to") String localFolder)
throws Exception {
final String basePath = HoodieCLI.getTableMetaClient().getBasePath();
@@ -81,7 +80,7 @@ public String exportInstants(
int numExports = limit == -1 ? Integer.MAX_VALUE : limit;
int numCopied = 0;
- if (! new File(localFolder).isDirectory()) {
+ if (!new File(localFolder).isDirectory()) {
throw new HoodieException(localFolder + " is not a valid local directory");
}
@@ -92,7 +91,7 @@ public String exportInstants(
// Archived instants are in the commit archive files
FileStatus[] statuses = FSUtils.getFs(basePath, HoodieCLI.conf).globStatus(archivePath);
- List archivedStatuses = Arrays.stream(statuses).sorted((f1, f2) -> (int)(f1.getModificationTime() - f2.getModificationTime())).collect(Collectors.toList());
+ List archivedStatuses = Arrays.stream(statuses).sorted((f1, f2) -> (int) (f1.getModificationTime() - f2.getModificationTime())).collect(Collectors.toList());
if (descending) {
Collections.reverse(nonArchivedInstants);
@@ -113,53 +112,55 @@ public String exportInstants(
private int copyArchivedInstants(List statuses, Set actionSet, int limit, String localFolder) throws Exception {
int copyCount = 0;
+ FileSystem fileSystem = FSUtils.getFs(HoodieCLI.getTableMetaClient().getBasePath(), HoodieCLI.conf);
for (FileStatus fs : statuses) {
// read the archived file
- Reader reader = HoodieLogFormat.newReader(FSUtils.getFs(HoodieCLI.getTableMetaClient().getBasePath(), HoodieCLI.conf),
- new HoodieLogFile(fs.getPath()), HoodieArchivedMetaEntry.getClassSchema());
+ Reader reader = HoodieLogFormat.newReader(fileSystem, new HoodieLogFile(fs.getPath()), HoodieArchivedMetaEntry.getClassSchema());
// read the avro blocks
while (reader.hasNext() && copyCount < limit) {
HoodieAvroDataBlock blk = (HoodieAvroDataBlock) reader.next();
- for (IndexedRecord ir : blk.getRecords()) {
- // Archived instants are saved as arvo encoded HoodieArchivedMetaEntry records. We need to get the
- // metadata record from the entry and convert it to json.
- HoodieArchivedMetaEntry archiveEntryRecord = (HoodieArchivedMetaEntry) SpecificData.get()
- .deepCopy(HoodieArchivedMetaEntry.SCHEMA$, ir);
-
- final String action = archiveEntryRecord.get("actionType").toString();
- if (!actionSet.contains(action)) {
- continue;
- }
-
- GenericRecord metadata = null;
- switch (action) {
- case HoodieTimeline.CLEAN_ACTION:
- metadata = archiveEntryRecord.getHoodieCleanMetadata();
- break;
- case HoodieTimeline.COMMIT_ACTION:
- case HoodieTimeline.DELTA_COMMIT_ACTION:
- metadata = archiveEntryRecord.getHoodieCommitMetadata();
- break;
- case HoodieTimeline.ROLLBACK_ACTION:
- metadata = archiveEntryRecord.getHoodieRollbackMetadata();
+ try (ClosableIterator recordItr = blk.getRecordIterator()) {
+ while (recordItr.hasNext()) {
+ IndexedRecord ir = recordItr.next();
+ // Archived instants are saved as arvo encoded HoodieArchivedMetaEntry records. We need to get the
+ // metadata record from the entry and convert it to json.
+ HoodieArchivedMetaEntry archiveEntryRecord = (HoodieArchivedMetaEntry) SpecificData.get()
+ .deepCopy(HoodieArchivedMetaEntry.SCHEMA$, ir);
+ final String action = archiveEntryRecord.get("actionType").toString();
+ if (!actionSet.contains(action)) {
+ continue;
+ }
+
+ GenericRecord metadata = null;
+ switch (action) {
+ case HoodieTimeline.CLEAN_ACTION:
+ metadata = archiveEntryRecord.getHoodieCleanMetadata();
+ break;
+ case HoodieTimeline.COMMIT_ACTION:
+ case HoodieTimeline.DELTA_COMMIT_ACTION:
+ metadata = archiveEntryRecord.getHoodieCommitMetadata();
+ break;
+ case HoodieTimeline.ROLLBACK_ACTION:
+ metadata = archiveEntryRecord.getHoodieRollbackMetadata();
+ break;
+ case HoodieTimeline.SAVEPOINT_ACTION:
+ metadata = archiveEntryRecord.getHoodieSavePointMetadata();
+ break;
+ case HoodieTimeline.COMPACTION_ACTION:
+ metadata = archiveEntryRecord.getHoodieCompactionMetadata();
+ break;
+ default:
+ throw new HoodieException("Unknown type of action " + action);
+ }
+
+ final String instantTime = archiveEntryRecord.get("commitTime").toString();
+ final String outPath = localFolder + Path.SEPARATOR + instantTime + "." + action;
+ writeToFile(outPath, HoodieAvroUtils.avroToJson(metadata, true));
+ if (++copyCount == limit) {
break;
- case HoodieTimeline.SAVEPOINT_ACTION:
- metadata = archiveEntryRecord.getHoodieSavePointMetadata();
- break;
- case HoodieTimeline.COMPACTION_ACTION:
- metadata = archiveEntryRecord.getHoodieCompactionMetadata();
- break;
- default:
- throw new HoodieException("Unknown type of action " + action);
- }
-
- final String instantTime = archiveEntryRecord.get("commitTime").toString();
- final String outPath = localFolder + Path.SEPARATOR + instantTime + "." + action;
- writeToFile(outPath, HoodieAvroUtils.avroToJson(metadata, true));
- if (++copyCount == limit) {
- break;
+ }
}
}
}
@@ -174,14 +175,13 @@ private int copyNonArchivedInstants(List instants, int limit, Str
int copyCount = 0;
if (instants.isEmpty()) {
- return limit;
+ return copyCount;
}
- final Logger LOG = LogManager.getLogger(ExportCommand.class);
final HoodieTableMetaClient metaClient = HoodieCLI.getTableMetaClient();
final HoodieActiveTimeline timeline = metaClient.getActiveTimeline();
for (HoodieInstant instant : instants) {
- String localPath = localFolder + File.separator + instant.getFileName();
+ String localPath = localFolder + Path.SEPARATOR + instant.getFileName();
byte[] data = null;
switch (instant.getAction()) {
@@ -217,6 +217,7 @@ private int copyNonArchivedInstants(List instants, int limit, Str
if (data != null) {
writeToFile(localPath, data);
+ copyCount = copyCount + 1;
}
}
diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/FileSystemViewCommand.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/FileSystemViewCommand.java
index ef76ee4e2f1ad..78e7d90195d58 100644
--- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/FileSystemViewCommand.java
+++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/FileSystemViewCommand.java
@@ -18,6 +18,9 @@
package org.apache.hudi.cli.commands;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
import org.apache.hudi.cli.HoodieCLI;
import org.apache.hudi.cli.HoodiePrintHelper;
import org.apache.hudi.cli.HoodieTableHeaderFields;
@@ -32,14 +35,9 @@
import org.apache.hudi.common.table.view.HoodieTableFileSystemView;
import org.apache.hudi.common.util.NumericUtils;
import org.apache.hudi.common.util.Option;
-
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.springframework.shell.core.CommandMarker;
-import org.springframework.shell.core.annotation.CliCommand;
-import org.springframework.shell.core.annotation.CliOption;
-import org.springframework.stereotype.Component;
+import org.springframework.shell.standard.ShellComponent;
+import org.springframework.shell.standard.ShellMethod;
+import org.springframework.shell.standard.ShellOption;
import java.io.IOException;
import java.io.Serializable;
@@ -55,30 +53,32 @@
/**
* CLI command to display file system options.
*/
-@Component
-public class FileSystemViewCommand implements CommandMarker {
+@ShellComponent
+public class FileSystemViewCommand {
- @CliCommand(value = "show fsview all", help = "Show entire file-system view")
+ @ShellMethod(key = "show fsview all", value = "Show entire file-system view")
public String showAllFileSlices(
- @CliOption(key = {"pathRegex"}, help = "regex to select files, eg: 2016/08/02",
- unspecifiedDefaultValue = "*/*/*") String globRegex,
- @CliOption(key = {"baseFileOnly"}, help = "Only display base files view",
- unspecifiedDefaultValue = "false") boolean baseFileOnly,
- @CliOption(key = {"maxInstant"}, help = "File-Slices upto this instant are displayed",
- unspecifiedDefaultValue = "") String maxInstant,
- @CliOption(key = {"includeMax"}, help = "Include Max Instant",
- unspecifiedDefaultValue = "false") boolean includeMaxInstant,
- @CliOption(key = {"includeInflight"}, help = "Include Inflight Instants",
- unspecifiedDefaultValue = "false") boolean includeInflight,
- @CliOption(key = {"excludeCompaction"}, help = "Exclude compaction Instants",
- unspecifiedDefaultValue = "false") boolean excludeCompaction,
- @CliOption(key = {"limit"}, help = "Limit rows to be displayed", unspecifiedDefaultValue = "-1") Integer limit,
- @CliOption(key = {"sortBy"}, help = "Sorting Field", unspecifiedDefaultValue = "") final String sortByField,
- @CliOption(key = {"desc"}, help = "Ordering", unspecifiedDefaultValue = "false") final boolean descending,
- @CliOption(key = {"headeronly"}, help = "Print Header Only",
- unspecifiedDefaultValue = "false") final boolean headerOnly)
+ @ShellOption(value = {"--pathRegex"}, help = "regex to select files, eg: par1",
+ defaultValue = "*") String globRegex,
+ @ShellOption(value = {"--baseFileOnly"}, help = "Only display base files view",
+ defaultValue = "false") boolean baseFileOnly,
+ @ShellOption(value = {"--maxInstant"}, help = "File-Slices upto this instant are displayed",
+ defaultValue = "") String maxInstant,
+ @ShellOption(value = {"--includeMax"}, help = "Include Max Instant",
+ defaultValue = "false") boolean includeMaxInstant,
+ @ShellOption(value = {"--includeInflight"}, help = "Include Inflight Instants",
+ defaultValue = "false") boolean includeInflight,
+ @ShellOption(value = {"--excludeCompaction"}, help = "Exclude compaction Instants",
+ defaultValue = "false") boolean excludeCompaction,
+ @ShellOption(value = {"--limit"}, help = "Limit rows to be displayed", defaultValue = "-1") Integer limit,
+ @ShellOption(value = {"--sortBy"}, help = "Sorting Field", defaultValue = "") final String sortByField,
+ @ShellOption(value = {"--desc"}, help = "Ordering", defaultValue = "false") final boolean descending,
+ @ShellOption(value = {"--headeronly"}, help = "Print Header Only",
+ defaultValue = "false") final boolean headerOnly)
throws IOException {
+ globRegex = globRegex == null ? "" : globRegex;
+
HoodieTableFileSystemView fsView = buildFileSystemView(globRegex, maxInstant, baseFileOnly, includeMaxInstant,
includeInflight, excludeCompaction);
List rows = new ArrayList<>();
@@ -117,26 +117,26 @@ public String showAllFileSlices(
return HoodiePrintHelper.print(header, fieldNameToConverterMap, sortByField, descending, limit, headerOnly, rows);
}
- @CliCommand(value = "show fsview latest", help = "Show latest file-system view")
+ @ShellMethod(key = "show fsview latest", value = "Show latest file-system view")
public String showLatestFileSlices(
- @CliOption(key = {"partitionPath"}, help = "A valid paritition path", mandatory = true) String partition,
- @CliOption(key = {"baseFileOnly"}, help = "Only display base file view",
- unspecifiedDefaultValue = "false") boolean baseFileOnly,
- @CliOption(key = {"maxInstant"}, help = "File-Slices upto this instant are displayed",
- unspecifiedDefaultValue = "") String maxInstant,
- @CliOption(key = {"merge"}, help = "Merge File Slices due to pending compaction",
- unspecifiedDefaultValue = "true") final boolean merge,
- @CliOption(key = {"includeMax"}, help = "Include Max Instant",
- unspecifiedDefaultValue = "false") boolean includeMaxInstant,
- @CliOption(key = {"includeInflight"}, help = "Include Inflight Instants",
- unspecifiedDefaultValue = "false") boolean includeInflight,
- @CliOption(key = {"excludeCompaction"}, help = "Exclude compaction Instants",
- unspecifiedDefaultValue = "false") boolean excludeCompaction,
- @CliOption(key = {"limit"}, help = "Limit rows to be displayed", unspecifiedDefaultValue = "-1") Integer limit,
- @CliOption(key = {"sortBy"}, help = "Sorting Field", unspecifiedDefaultValue = "") final String sortByField,
- @CliOption(key = {"desc"}, help = "Ordering", unspecifiedDefaultValue = "false") final boolean descending,
- @CliOption(key = {"headeronly"}, help = "Print Header Only",
- unspecifiedDefaultValue = "false") final boolean headerOnly)
+ @ShellOption(value = {"--partitionPath"}, help = "A valid partition path", defaultValue = "") String partition,
+ @ShellOption(value = {"--baseFileOnly"}, help = "Only display base file view",
+ defaultValue = "false") boolean baseFileOnly,
+ @ShellOption(value = {"--maxInstant"}, help = "File-Slices upto this instant are displayed",
+ defaultValue = "") String maxInstant,
+ @ShellOption(value = {"--merge"}, help = "Merge File Slices due to pending compaction",
+ defaultValue = "true") final boolean merge,
+ @ShellOption(value = {"--includeMax"}, help = "Include Max Instant",
+ defaultValue = "false") boolean includeMaxInstant,
+ @ShellOption(value = {"--includeInflight"}, help = "Include Inflight Instants",
+ defaultValue = "false") boolean includeInflight,
+ @ShellOption(value = {"--excludeCompaction"}, help = "Exclude compaction Instants",
+ defaultValue = "false") boolean excludeCompaction,
+ @ShellOption(value = {"--limit"}, help = "Limit rows to be displayed", defaultValue = "-1") Integer limit,
+ @ShellOption(value = {"--sortBy"}, help = "Sorting Field", defaultValue = "") final String sortByField,
+ @ShellOption(value = {"--desc"}, help = "Ordering", defaultValue = "false") final boolean descending,
+ @ShellOption(value = {"--headeronly"}, help = "Print Header Only",
+ defaultValue = "false") final boolean headerOnly)
throws IOException {
HoodieTableFileSystemView fsView = buildFileSystemView(partition, maxInstant, baseFileOnly, includeMaxInstant,
@@ -223,21 +223,21 @@ public String showLatestFileSlices(
/**
* Build File System View.
- *
- * @param globRegex Path Regex
- * @param maxInstant Max Instants to be used for displaying file-instants
- * @param basefileOnly Include only base file view
+ *
+ * @param globRegex Path Regex
+ * @param maxInstant Max Instants to be used for displaying file-instants
+ * @param basefileOnly Include only base file view
* @param includeMaxInstant Include Max instant
- * @param includeInflight Include inflight instants
+ * @param includeInflight Include inflight instants
* @param excludeCompaction Exclude Compaction instants
* @return
* @throws IOException
*/
private HoodieTableFileSystemView buildFileSystemView(String globRegex, String maxInstant, boolean basefileOnly,
- boolean includeMaxInstant, boolean includeInflight, boolean excludeCompaction) throws IOException {
+ boolean includeMaxInstant, boolean includeInflight, boolean excludeCompaction) throws IOException {
HoodieTableMetaClient client = HoodieCLI.getTableMetaClient();
HoodieTableMetaClient metaClient =
- new HoodieTableMetaClient(client.getHadoopConf(), client.getBasePath(), true);
+ HoodieTableMetaClient.builder().setConf(client.getHadoopConf()).setBasePath(client.getBasePath()).setLoadActiveTimelineOnLoad(true).build();
FileSystem fs = HoodieCLI.fs;
String globPath = String.format("%s/%s/*", client.getBasePath(), globRegex);
List statuses = FSUtils.getGlobStatusExcludingMetaFolder(fs, new Path(globPath));
@@ -249,7 +249,7 @@ private HoodieTableFileSystemView buildFileSystemView(String globRegex, String m
} else if (excludeCompaction) {
timeline = metaClient.getActiveTimeline().getCommitsTimeline();
} else {
- timeline = metaClient.getActiveTimeline().getCommitsAndCompactionTimeline();
+ timeline = metaClient.getActiveTimeline().getWriteTimeline();
}
if (!includeInflight) {
diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/HDFSParquetImportCommand.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/HDFSParquetImportCommand.java
index a31f31012830d..9ea5bbab04bda 100644
--- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/HDFSParquetImportCommand.java
+++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/HDFSParquetImportCommand.java
@@ -23,46 +23,47 @@
import org.apache.hudi.cli.utils.SparkUtil;
import org.apache.hudi.utilities.HDFSParquetImporter.FormatValidator;
import org.apache.hudi.utilities.UtilHelpers;
-
+import org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer;
import org.apache.spark.launcher.SparkLauncher;
import org.apache.spark.util.Utils;
-import org.springframework.shell.core.CommandMarker;
-import org.springframework.shell.core.annotation.CliCommand;
-import org.springframework.shell.core.annotation.CliOption;
-import org.springframework.stereotype.Component;
-
+import org.springframework.shell.standard.ShellComponent;
+import org.springframework.shell.standard.ShellMethod;
+import org.springframework.shell.standard.ShellOption;
import scala.collection.JavaConverters;
/**
* CLI command for importing parquet table to hudi table.
+ *
+ * @see HoodieDeltaStreamer
+ * @deprecated This utility is deprecated in 0.10.0 and will be removed in 0.11.0. Use {@link HoodieDeltaStreamer.Config#runBootstrap} instead.
*/
-@Component
-public class HDFSParquetImportCommand implements CommandMarker {
+@ShellComponent
+public class HDFSParquetImportCommand {
- @CliCommand(value = "hdfsparquetimport", help = "Imports Parquet table to a hoodie table")
+ @ShellMethod(key = "hdfsparquetimport", value = "Imports Parquet table to a hoodie table")
public String convert(
- @CliOption(key = "upsert", unspecifiedDefaultValue = "false",
+ @ShellOption(value = "--upsert", defaultValue = "false",
help = "Uses upsert API instead of the default insert API of WriteClient") boolean useUpsert,
- @CliOption(key = "srcPath", mandatory = true, help = "Base path for the input table") final String srcPath,
- @CliOption(key = "targetPath", mandatory = true,
+ @ShellOption(value = "--srcPath", help = "Base path for the input table") final String srcPath,
+ @ShellOption(value = "--targetPath",
help = "Base path for the target hoodie table") final String targetPath,
- @CliOption(key = "tableName", mandatory = true, help = "Table name") final String tableName,
- @CliOption(key = "tableType", mandatory = true, help = "Table type") final String tableType,
- @CliOption(key = "rowKeyField", mandatory = true, help = "Row key field name") final String rowKeyField,
- @CliOption(key = "partitionPathField", mandatory = true,
+ @ShellOption(value = "--tableName", help = "Table name") final String tableName,
+ @ShellOption(value = "--tableType", help = "Table type") final String tableType,
+ @ShellOption(value = "--rowKeyField", help = "Row key field name") final String rowKeyField,
+ @ShellOption(value = "--partitionPathField", defaultValue = "",
help = "Partition path field name") final String partitionPathField,
- @CliOption(key = {"parallelism"}, mandatory = true,
+ @ShellOption(value = {"--parallelism"},
help = "Parallelism for hoodie insert") final String parallelism,
- @CliOption(key = "schemaFilePath", mandatory = true,
+ @ShellOption(value = "--schemaFilePath",
help = "Path for Avro schema file") final String schemaFilePath,
- @CliOption(key = "format", mandatory = true, help = "Format for the input data") final String format,
- @CliOption(key = "sparkMaster", unspecifiedDefaultValue = "", help = "Spark Master") String master,
- @CliOption(key = "sparkMemory", mandatory = true, help = "Spark executor memory") final String sparkMemory,
- @CliOption(key = "retry", mandatory = true, help = "Number of retries") final String retry,
- @CliOption(key = "propsFilePath", help = "path to properties file on localfs or dfs with configurations for hoodie client for importing",
- unspecifiedDefaultValue = "") final String propsFilePath,
- @CliOption(key = "hoodieConfigs", help = "Any configuration that can be set in the properties file can be passed here in the form of an array",
- unspecifiedDefaultValue = "") final String[] configs) throws Exception {
+ @ShellOption(value = "--format", help = "Format for the input data") final String format,
+ @ShellOption(value = "--sparkMaster", defaultValue = "", help = "Spark Master") String master,
+ @ShellOption(value = "--sparkMemory", help = "Spark executor memory") final String sparkMemory,
+ @ShellOption(value = "--retry", help = "Number of retries") final String retry,
+ @ShellOption(value = "--propsFilePath", help = "path to properties file on localfs or dfs with configurations for hoodie client for importing",
+ defaultValue = "") final String propsFilePath,
+ @ShellOption(value = "--hoodieConfigs", help = "Any configuration that can be set in the properties file can be passed here in the form of an array",
+ defaultValue = "") final String[] configs) throws Exception {
(new FormatValidator()).validate("format", format);
diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/HoodieLogFileCommand.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/HoodieLogFileCommand.java
index e53dd38891604..56e00aa24cd7c 100644
--- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/HoodieLogFileCommand.java
+++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/HoodieLogFileCommand.java
@@ -18,10 +18,17 @@
package org.apache.hudi.cli.commands;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.avro.Schema;
+import org.apache.avro.generic.IndexedRecord;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
import org.apache.hudi.cli.HoodieCLI;
import org.apache.hudi.cli.HoodiePrintHelper;
import org.apache.hudi.cli.HoodieTableHeaderFields;
import org.apache.hudi.cli.TableHeader;
+import org.apache.hudi.common.config.HoodieCommonConfig;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.HoodieLogFile;
import org.apache.hudi.common.model.HoodieRecord;
@@ -36,21 +43,16 @@
import org.apache.hudi.common.table.log.block.HoodieLogBlock;
import org.apache.hudi.common.table.log.block.HoodieLogBlock.HeaderMetadataType;
import org.apache.hudi.common.table.log.block.HoodieLogBlock.HoodieLogBlockType;
+import org.apache.hudi.common.util.ClosableIterator;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.config.HoodieCompactionConfig;
import org.apache.hudi.config.HoodieMemoryConfig;
-
-import com.fasterxml.jackson.databind.ObjectMapper;
-import org.apache.avro.Schema;
-import org.apache.avro.generic.IndexedRecord;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
import org.apache.parquet.avro.AvroSchemaConverter;
-import org.springframework.shell.core.CommandMarker;
-import org.springframework.shell.core.annotation.CliCommand;
-import org.springframework.shell.core.annotation.CliOption;
-import org.springframework.stereotype.Component;
+import org.springframework.shell.standard.ShellComponent;
+import org.springframework.shell.standard.ShellMethod;
+import org.springframework.shell.standard.ShellOption;
+import scala.Tuple2;
+import scala.Tuple3;
import java.io.IOException;
import java.util.ArrayList;
@@ -59,26 +61,26 @@
import java.util.List;
import java.util.Map;
import java.util.Objects;
+import java.util.concurrent.atomic.AtomicInteger;
import java.util.stream.Collectors;
-import scala.Tuple2;
-import scala.Tuple3;
+import static org.apache.hudi.common.util.ValidationUtils.checkArgument;
/**
* CLI command to display log file options.
*/
-@Component
-public class HoodieLogFileCommand implements CommandMarker {
+@ShellComponent
+public class HoodieLogFileCommand {
- @CliCommand(value = "show logfile metadata", help = "Read commit metadata from log files")
+ @ShellMethod(key = "show logfile metadata", value = "Read commit metadata from log files")
public String showLogFileCommits(
- @CliOption(key = "logFilePathPattern", mandatory = true,
+ @ShellOption(value = "--logFilePathPattern",
help = "Fully qualified path for the log file") final String logFilePathPattern,
- @CliOption(key = {"limit"}, help = "Limit commits", unspecifiedDefaultValue = "-1") final Integer limit,
- @CliOption(key = {"sortBy"}, help = "Sorting Field", unspecifiedDefaultValue = "") final String sortByField,
- @CliOption(key = {"desc"}, help = "Ordering", unspecifiedDefaultValue = "false") final boolean descending,
- @CliOption(key = {"headeronly"}, help = "Print Header Only",
- unspecifiedDefaultValue = "false") final boolean headerOnly)
+ @ShellOption(value = {"--limit"}, help = "Limit commits", defaultValue = "-1") final Integer limit,
+ @ShellOption(value = {"--sortBy"}, help = "Sorting Field", defaultValue = "") final String sortByField,
+ @ShellOption(value = {"--desc"}, help = "Ordering", defaultValue = "false") final boolean descending,
+ @ShellOption(value = {"--headeronly"}, help = "Print Header Only",
+ defaultValue = "false") final boolean headerOnly)
throws IOException {
FileSystem fs = HoodieCLI.getTableMetaClient().getFs();
@@ -99,7 +101,7 @@ public String showLogFileCommits(
while (reader.hasNext()) {
HoodieLogBlock n = reader.next();
String instantTime;
- int recordCount = 0;
+ AtomicInteger recordCount = new AtomicInteger(0);
if (n instanceof HoodieCorruptBlock) {
try {
instantTime = n.getLogBlockHeader().get(HeaderMetadataType.INSTANT_TIME);
@@ -119,17 +121,19 @@ public String showLogFileCommits(
instantTime = "dummy_instant_time_" + dummyInstantTimeCount;
}
if (n instanceof HoodieDataBlock) {
- recordCount = ((HoodieDataBlock) n).getRecords().size();
+ try (ClosableIterator recordItr = ((HoodieDataBlock) n).getRecordIterator()) {
+ recordItr.forEachRemaining(r -> recordCount.incrementAndGet());
+ }
}
}
if (commitCountAndMetadata.containsKey(instantTime)) {
commitCountAndMetadata.get(instantTime).add(
- new Tuple3<>(n.getBlockType(), new Tuple2<>(n.getLogBlockHeader(), n.getLogBlockFooter()), recordCount));
+ new Tuple3<>(n.getBlockType(), new Tuple2<>(n.getLogBlockHeader(), n.getLogBlockFooter()), recordCount.get()));
} else {
List, Map>, Integer>> list =
new ArrayList<>();
list.add(
- new Tuple3<>(n.getBlockType(), new Tuple2<>(n.getLogBlockHeader(), n.getLogBlockFooter()), recordCount));
+ new Tuple3<>(n.getBlockType(), new Tuple2<>(n.getLogBlockHeader(), n.getLogBlockFooter()), recordCount.get()));
commitCountAndMetadata.put(instantTime, list);
}
}
@@ -161,14 +165,14 @@ public String showLogFileCommits(
return HoodiePrintHelper.print(header, new HashMap<>(), sortByField, descending, limit, headerOnly, rows);
}
- @CliCommand(value = "show logfile records", help = "Read records from log files")
+ @ShellMethod(key = "show logfile records", value = "Read records from log files")
public String showLogFileRecords(
- @CliOption(key = {"limit"}, help = "Limit commits",
- unspecifiedDefaultValue = "10") final Integer limit,
- @CliOption(key = "logFilePathPattern", mandatory = true,
+ @ShellOption(value = {"--limit"}, help = "Limit commits",
+ defaultValue = "10") final Integer limit,
+ @ShellOption(value = "--logFilePathPattern",
help = "Fully qualified paths for the log files") final String logFilePathPattern,
- @CliOption(key = "mergeRecords", help = "If the records in the log files should be merged",
- unspecifiedDefaultValue = "false") final Boolean shouldMerge)
+ @ShellOption(value = "--mergeRecords", help = "If the records in the log files should be merged",
+ defaultValue = "false") final Boolean shouldMerge)
throws IOException {
System.out.println("===============> Showing only " + limit + " records <===============");
@@ -180,7 +184,7 @@ public String showLogFileRecords(
.collect(Collectors.toList());
// logFilePaths size must > 1
- assert logFilePaths.size() > 0 : "There is no log file";
+ checkArgument(logFilePaths.size() > 0, "There is no log file");
// TODO : readerSchema can change across blocks/log files, fix this inside Scanner
AvroSchemaConverter converter = new AvroSchemaConverter();
@@ -203,14 +207,16 @@ public String showLogFileRecords(
.getCommitTimeline().lastInstant().get().getTimestamp())
.withReadBlocksLazily(
Boolean.parseBoolean(
- HoodieCompactionConfig.DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED))
+ HoodieCompactionConfig.COMPACTION_LAZY_BLOCK_READ_ENABLE.defaultValue()))
.withReverseReader(
Boolean.parseBoolean(
- HoodieCompactionConfig.DEFAULT_COMPACTION_REVERSE_LOG_READ_ENABLED))
- .withBufferSize(HoodieMemoryConfig.DEFAULT_MAX_DFS_STREAM_BUFFER_SIZE)
+ HoodieCompactionConfig.COMPACTION_REVERSE_LOG_READ_ENABLE.defaultValue()))
+ .withBufferSize(HoodieMemoryConfig.MAX_DFS_STREAM_BUFFER_SIZE.defaultValue())
.withMaxMemorySizeInBytes(
HoodieMemoryConfig.DEFAULT_MAX_MEMORY_FOR_SPILLABLE_MAP_IN_BYTES)
- .withSpillableMapBasePath(HoodieMemoryConfig.DEFAULT_SPILLABLE_MAP_BASE_PATH)
+ .withSpillableMapBasePath(HoodieMemoryConfig.SPILLABLE_MAP_BASE_PATH.defaultValue())
+ .withDiskMapType(HoodieCommonConfig.SPILLABLE_DISK_MAP_TYPE.defaultValue())
+ .withBitCaskDiskMapCompressionEnabled(HoodieCommonConfig.DISK_MAP_BITCASK_COMPRESSION_ENABLED.defaultValue())
.build();
for (HoodieRecord extends HoodieRecordPayload> hoodieRecord : scanner) {
Option record = hoodieRecord.getData().getInsertValue(readerSchema);
@@ -229,11 +235,12 @@ public String showLogFileRecords(
HoodieLogBlock n = reader.next();
if (n instanceof HoodieDataBlock) {
HoodieDataBlock blk = (HoodieDataBlock) n;
- List records = blk.getRecords();
- for (IndexedRecord record : records) {
- if (allRecords.size() < limit) {
- allRecords.add(record);
- }
+ try (ClosableIterator recordItr = blk.getRecordIterator()) {
+ recordItr.forEachRemaining(record -> {
+ if (allRecords.size() < limit) {
+ allRecords.add(record);
+ }
+ });
}
}
}
diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/HoodieSyncCommand.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/HoodieSyncCommand.java
deleted file mode 100644
index 66c2eb02159e4..0000000000000
--- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/HoodieSyncCommand.java
+++ /dev/null
@@ -1,107 +0,0 @@
-/*
- * 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.cli.commands;
-
-import org.apache.hudi.cli.HoodieCLI;
-import org.apache.hudi.cli.utils.CommitUtil;
-import org.apache.hudi.cli.utils.HiveUtil;
-import org.apache.hudi.common.table.HoodieTableMetaClient;
-import org.apache.hudi.common.table.timeline.HoodieInstant;
-import org.apache.hudi.common.table.timeline.HoodieTimeline;
-import org.apache.hudi.exception.HoodieException;
-
-import org.springframework.shell.core.CommandMarker;
-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.List;
-import java.util.stream.Collectors;
-
-/**
- * CLI command to display sync options.
- */
-@Component
-public class HoodieSyncCommand implements CommandMarker {
-
- @CliCommand(value = "sync validate", help = "Validate the sync by counting the number of records")
- public String validateSync(
- @CliOption(key = {"mode"}, unspecifiedDefaultValue = "complete", help = "Check mode") final String mode,
- @CliOption(key = {"sourceDb"}, unspecifiedDefaultValue = "rawdata", help = "source database") final String srcDb,
- @CliOption(key = {"targetDb"}, unspecifiedDefaultValue = "dwh_hoodie",
- help = "target database") final String tgtDb,
- @CliOption(key = {"partitionCount"}, unspecifiedDefaultValue = "5",
- help = "total number of recent partitions to validate") final int partitionCount,
- @CliOption(key = {"hiveServerUrl"}, mandatory = true,
- help = "hiveServerURL to connect to") final String hiveServerUrl,
- @CliOption(key = {"hiveUser"}, unspecifiedDefaultValue = "",
- help = "hive username to connect to") final String hiveUser,
- @CliOption(key = {"hivePass"}, mandatory = true, unspecifiedDefaultValue = "",
- help = "hive password to connect to") final String hivePass)
- throws Exception {
- if (HoodieCLI.syncTableMetadata == null) {
- throw new HoodieException("Sync validate request target table not null.");
- }
- HoodieTableMetaClient target = HoodieCLI.syncTableMetadata;
- HoodieTimeline targetTimeline = target.getActiveTimeline().getCommitsTimeline();
- HoodieTableMetaClient source = HoodieCLI.getTableMetaClient();
- HoodieTimeline sourceTimeline = source.getActiveTimeline().getCommitsTimeline();
- long sourceCount = 0;
- long targetCount = 0;
- if ("complete".equals(mode)) {
- sourceCount = HiveUtil.countRecords(hiveServerUrl, source, srcDb, hiveUser, hivePass);
- targetCount = HiveUtil.countRecords(hiveServerUrl, target, tgtDb, hiveUser, hivePass);
- } else if ("latestPartitions".equals(mode)) {
- sourceCount = HiveUtil.countRecords(hiveServerUrl, source, srcDb, partitionCount, hiveUser, hivePass);
- targetCount = HiveUtil.countRecords(hiveServerUrl, target, tgtDb, partitionCount, hiveUser, hivePass);
- }
-
- String targetLatestCommit =
- targetTimeline.getInstants().iterator().hasNext() ? targetTimeline.lastInstant().get().getTimestamp() : "0";
- String sourceLatestCommit =
- sourceTimeline.getInstants().iterator().hasNext() ? sourceTimeline.lastInstant().get().getTimestamp() : "0";
-
- if (sourceLatestCommit != null
- && HoodieTimeline.compareTimestamps(targetLatestCommit, HoodieTimeline.GREATER_THAN, sourceLatestCommit)) {
- // source is behind the target
- return getString(target, targetTimeline, source, sourceCount, targetCount, sourceLatestCommit);
- } else {
- return getString(source, sourceTimeline, target, targetCount, sourceCount, targetLatestCommit);
-
- }
- }
-
- private String getString(HoodieTableMetaClient target, HoodieTimeline targetTimeline, HoodieTableMetaClient source, long sourceCount, long targetCount, String sourceLatestCommit)
- throws IOException {
- List commitsToCatchup = targetTimeline.findInstantsAfter(sourceLatestCommit, Integer.MAX_VALUE)
- .getInstants().collect(Collectors.toList());
- if (commitsToCatchup.isEmpty()) {
- return "Count difference now is (count(" + target.getTableConfig().getTableName() + ") - count("
- + source.getTableConfig().getTableName() + ") == " + (targetCount - sourceCount);
- } else {
- long newInserts = CommitUtil.countNewRecords(target,
- commitsToCatchup.stream().map(HoodieInstant::getTimestamp).collect(Collectors.toList()));
- return "Count difference now is (count(" + target.getTableConfig().getTableName() + ") - count("
- + source.getTableConfig().getTableName() + ") == " + (targetCount - sourceCount) + ". Catch up count is "
- + newInserts;
- }
- }
-
-}
diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/HoodieSyncValidateCommand.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/HoodieSyncValidateCommand.java
new file mode 100644
index 0000000000000..0fc26a55b8990
--- /dev/null
+++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/HoodieSyncValidateCommand.java
@@ -0,0 +1,106 @@
+/*
+ * 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.cli.commands;
+
+import org.apache.hudi.cli.HoodieCLI;
+import org.apache.hudi.cli.utils.HiveUtil;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.common.table.timeline.HoodieTimeline;
+import org.apache.hudi.exception.HoodieException;
+import org.springframework.shell.standard.ShellComponent;
+import org.springframework.shell.standard.ShellMethod;
+import org.springframework.shell.standard.ShellOption;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.stream.Collectors;
+
+import static org.apache.hudi.cli.utils.CommitUtil.countNewRecords;
+
+/**
+ * CLI command to display sync options.
+ */
+@ShellComponent
+public class HoodieSyncValidateCommand {
+
+ @ShellMethod(key = "sync validate", value = "Validate the sync by counting the number of records")
+ public String validateSync(
+ @ShellOption(value = {"--mode"}, defaultValue = "complete", help = "Check mode") final String mode,
+ @ShellOption(value = {"--sourceDb"}, defaultValue = "rawdata", help = "source database") final String srcDb,
+ @ShellOption(value = {"--targetDb"}, defaultValue = "dwh_hoodie",
+ help = "target database") final String tgtDb,
+ @ShellOption(value = {"--partitionCount"}, defaultValue = "5",
+ help = "total number of recent partitions to validate") final int partitionCount,
+ @ShellOption(value = {"--hiveServerUrl"},
+ help = "hiveServerURL to connect to") final String hiveServerUrl,
+ @ShellOption(value = {"--hiveUser"}, defaultValue = "",
+ help = "hive username to connect to") final String hiveUser,
+ @ShellOption(value = {"--hivePass"}, defaultValue = "",
+ help = "hive password to connect to") final String hivePass)
+ throws Exception {
+ if (HoodieCLI.syncTableMetadata == null) {
+ throw new HoodieException("Sync validate request target table not null.");
+ }
+ HoodieTableMetaClient target = HoodieCLI.syncTableMetadata;
+ HoodieTimeline targetTimeline = target.getActiveTimeline().getCommitsTimeline();
+ HoodieTableMetaClient source = HoodieCLI.getTableMetaClient();
+ HoodieTimeline sourceTimeline = source.getActiveTimeline().getCommitsTimeline();
+ long sourceCount = 0;
+ long targetCount = 0;
+ if ("complete".equals(mode)) {
+ sourceCount = HiveUtil.countRecords(hiveServerUrl, source, srcDb, hiveUser, hivePass);
+ targetCount = HiveUtil.countRecords(hiveServerUrl, target, tgtDb, hiveUser, hivePass);
+ } else if ("latestPartitions".equals(mode)) {
+ sourceCount = HiveUtil.countRecords(hiveServerUrl, source, srcDb, partitionCount, hiveUser, hivePass);
+ targetCount = HiveUtil.countRecords(hiveServerUrl, target, tgtDb, partitionCount, hiveUser, hivePass);
+ }
+
+ String targetLatestCommit =
+ targetTimeline.getInstants().iterator().hasNext() ? targetTimeline.lastInstant().get().getTimestamp() : "0";
+ String sourceLatestCommit =
+ sourceTimeline.getInstants().iterator().hasNext() ? sourceTimeline.lastInstant().get().getTimestamp() : "0";
+
+ if (sourceLatestCommit != null
+ && HoodieTimeline.compareTimestamps(targetLatestCommit, HoodieTimeline.GREATER_THAN, sourceLatestCommit)) {
+ // source is behind the target
+ return getString(target, targetTimeline, source, sourceCount, targetCount, sourceLatestCommit);
+ } else {
+ return getString(source, sourceTimeline, target, targetCount, sourceCount, targetLatestCommit);
+
+ }
+ }
+
+ private String getString(HoodieTableMetaClient target, HoodieTimeline targetTimeline, HoodieTableMetaClient source, long sourceCount, long targetCount, String sourceLatestCommit)
+ throws IOException {
+ List commitsToCatchup = targetTimeline.findInstantsAfter(sourceLatestCommit, Integer.MAX_VALUE)
+ .getInstants().collect(Collectors.toList());
+ if (commitsToCatchup.isEmpty()) {
+ return "Count difference now is (count(" + target.getTableConfig().getTableName() + ") - count("
+ + source.getTableConfig().getTableName() + ") == " + (targetCount - sourceCount);
+ } else {
+ long newInserts = countNewRecords(target,
+ commitsToCatchup.stream().map(HoodieInstant::getTimestamp).collect(Collectors.toList()));
+ return "Count difference now is (count(" + target.getTableConfig().getTableName() + ") - count("
+ + source.getTableConfig().getTableName() + ") == " + (targetCount - sourceCount) + ". Catch up count is "
+ + newInserts;
+ }
+ }
+
+}
diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/KerberosAuthenticationCommand.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/KerberosAuthenticationCommand.java
new file mode 100644
index 0000000000000..d79279a315809
--- /dev/null
+++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/KerberosAuthenticationCommand.java
@@ -0,0 +1,60 @@
+/*
+ * 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.cli.commands;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.springframework.shell.standard.ShellComponent;
+import org.springframework.shell.standard.ShellMethod;
+import org.springframework.shell.standard.ShellOption;
+
+import java.io.IOException;
+
+/**
+ * CLI command to perform Kerberos authentication.
+ */
+@ShellComponent
+public class KerberosAuthenticationCommand {
+
+ @ShellMethod(key = "kerberos kinit", value = "Perform Kerberos authentication")
+ public String performKerberosAuthentication(
+ @ShellOption(value = "--krb5conf", help = "Path to krb5.conf", defaultValue = "/etc/krb5.conf") String krb5ConfPath,
+ @ShellOption(value = "--principal", help = "Kerberos principal") String principal,
+ @ShellOption(value = "--keytab", help = "Path to keytab") String keytabPath) throws IOException {
+
+ System.out.println("Perform Kerberos authentication");
+ System.out.println("Parameters:");
+ System.out.println("--krb5conf: " + krb5ConfPath);
+ System.out.println("--principal: " + principal);
+ System.out.println("--keytab: " + keytabPath);
+
+ System.setProperty("java.security.krb5.conf", krb5ConfPath);
+ Configuration conf = new Configuration();
+ conf.set("hadoop.security.authentication", "kerberos");
+ conf.set("keytab.file", keytabPath);
+ conf.set("kerberos.principal", principal);
+ UserGroupInformation.setConfiguration(conf);
+ UserGroupInformation.loginUserFromKeytab(principal, keytabPath);
+
+ System.out.println("Kerberos current user: " + UserGroupInformation.getCurrentUser());
+ System.out.println("Kerberos login user: " + UserGroupInformation.getLoginUser());
+
+ return "Kerberos authentication success";
+ }
+}
diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/MarkersCommand.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/MarkersCommand.java
new file mode 100644
index 0000000000000..008c61aa9a84b
--- /dev/null
+++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/MarkersCommand.java
@@ -0,0 +1,59 @@
+/*
+ * 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.cli.commands;
+
+import org.apache.hudi.cli.HoodieCLI;
+import org.apache.hudi.cli.utils.InputStreamConsumer;
+import org.apache.hudi.cli.utils.SparkUtil;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.spark.launcher.SparkLauncher;
+import org.springframework.shell.standard.ShellComponent;
+import org.springframework.shell.standard.ShellMethod;
+import org.springframework.shell.standard.ShellOption;
+
+/**
+ * CLI command for marker options.
+ */
+@ShellComponent
+public class MarkersCommand {
+
+ @ShellMethod(key = "marker delete", value = "Delete the marker")
+ public String deleteMarker(
+ @ShellOption(value = {"--commit"}, help = "Delete a marker") final String instantTime,
+ @ShellOption(value = {"--sparkProperties"}, help = "Spark Properties File Path",
+ defaultValue = "") final String sparkPropertiesPath,
+ @ShellOption(value = "--sparkMaster", defaultValue = "", help = "Spark Master") String master,
+ @ShellOption(value = "--sparkMemory", defaultValue = "1G",
+ help = "Spark executor memory") final String sparkMemory)
+ throws Exception {
+ HoodieTableMetaClient metaClient = HoodieCLI.getTableMetaClient();
+ SparkLauncher sparkLauncher = SparkUtil.initLauncher(sparkPropertiesPath);
+ sparkLauncher.addAppArgs(SparkMain.SparkCommand.DELETE_MARKER.toString(), master, sparkMemory, instantTime,
+ metaClient.getBasePath());
+ Process process = sparkLauncher.launch();
+ InputStreamConsumer.captureOutput(process);
+ int exitCode = process.waitFor();
+ // Refresh the current
+ HoodieCLI.refreshTableMetadata();
+ if (exitCode != 0) {
+ return String.format("Failed: Could not delete marker \"%s\".", instantTime);
+ }
+ return String.format("Marker \"%s\" deleted.", instantTime);
+ }
+}
diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/MetadataCommand.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/MetadataCommand.java
new file mode 100644
index 0000000000000..65b01bb2545e4
--- /dev/null
+++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/MetadataCommand.java
@@ -0,0 +1,374 @@
+/*
+ * 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.cli.commands;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hudi.cli.HoodieCLI;
+import org.apache.hudi.cli.HoodiePrintHelper;
+import org.apache.hudi.cli.TableHeader;
+import org.apache.hudi.cli.utils.SparkUtil;
+import org.apache.hudi.client.common.HoodieSparkEngineContext;
+import org.apache.hudi.common.config.HoodieMetadataConfig;
+import org.apache.hudi.common.engine.HoodieLocalEngineContext;
+import org.apache.hudi.common.util.HoodieTimer;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.common.util.ValidationUtils;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.metadata.HoodieBackedTableMetadata;
+import org.apache.hudi.metadata.HoodieTableMetadata;
+import org.apache.hudi.metadata.SparkHoodieBackedTableMetadataWriter;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.springframework.shell.standard.ShellComponent;
+import org.springframework.shell.standard.ShellMethod;
+import org.springframework.shell.standard.ShellOption;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * CLI commands to operate on the Metadata Table.
+ *
+ *
+ * Example:
+ * The default spark.master conf is set to yarn. If you are running on a local deployment,
+ * we can set the spark master to local using set conf command.
+ * > set --conf SPARK_MASTER=local[2]
+ *
+ * Connect to the table
+ * > connect --path {path to hudi table}
+ *
+ * Run metadata commands
+ * > metadata list-partitions
+ */
+@ShellComponent
+public class MetadataCommand {
+
+ private static final Logger LOG = LogManager.getLogger(MetadataCommand.class);
+ private static String metadataBaseDirectory;
+ private JavaSparkContext jsc;
+
+ /**
+ * Sets the directory to store/read Metadata Table.
+ *
+ * This can be used to store the metadata table away from the dataset directory.
+ * - Useful for testing as well as for using via the HUDI CLI so that the actual dataset is not written to.
+ * - Useful for testing Metadata Table performance and operations on existing datasets before enabling.
+ */
+ public static void setMetadataBaseDirectory(String metadataDir) {
+ ValidationUtils.checkState(metadataBaseDirectory == null,
+ "metadataBaseDirectory is already set to " + metadataBaseDirectory);
+ metadataBaseDirectory = metadataDir;
+ }
+
+ public static String getMetadataTableBasePath(String tableBasePath) {
+ if (metadataBaseDirectory != null) {
+ return metadataBaseDirectory;
+ }
+ return HoodieTableMetadata.getMetadataTableBasePath(tableBasePath);
+ }
+
+ @ShellMethod(key = "metadata set", value = "Set options for Metadata Table")
+ public String set(@ShellOption(value = {"--metadataDir"},
+ help = "Directory to read/write metadata table (can be different from dataset)", defaultValue = "") final String metadataDir) {
+ if (!metadataDir.isEmpty()) {
+ setMetadataBaseDirectory(metadataDir);
+ }
+
+ return "Ok";
+ }
+
+ @ShellMethod(key = "metadata create", value = "Create the Metadata Table if it does not exist")
+ public String create(
+ @ShellOption(value = "--sparkMaster", defaultValue = SparkUtil.DEFAULT_SPARK_MASTER, help = "Spark master") final String master
+ ) throws IOException {
+ HoodieCLI.getTableMetaClient();
+ Path metadataPath = new Path(getMetadataTableBasePath(HoodieCLI.basePath));
+ try {
+ FileStatus[] statuses = HoodieCLI.fs.listStatus(metadataPath);
+ if (statuses.length > 0) {
+ throw new RuntimeException("Metadata directory (" + metadataPath.toString() + ") not empty.");
+ }
+ } catch (FileNotFoundException e) {
+ // Metadata directory does not exist yet
+ HoodieCLI.fs.mkdirs(metadataPath);
+ }
+
+ HoodieTimer timer = new HoodieTimer().startTimer();
+ HoodieWriteConfig writeConfig = getWriteConfig();
+ initJavaSparkContext(Option.of(master));
+ SparkHoodieBackedTableMetadataWriter.create(HoodieCLI.conf, writeConfig, new HoodieSparkEngineContext(jsc));
+ return String.format("Created Metadata Table in %s (duration=%.2f secs)", metadataPath, timer.endTimer() / 1000.0);
+ }
+
+ @ShellMethod(key = "metadata delete", value = "Remove the Metadata Table")
+ public String delete() throws Exception {
+ HoodieCLI.getTableMetaClient();
+ Path metadataPath = new Path(getMetadataTableBasePath(HoodieCLI.basePath));
+ try {
+ FileStatus[] statuses = HoodieCLI.fs.listStatus(metadataPath);
+ if (statuses.length > 0) {
+ HoodieCLI.fs.delete(metadataPath, true);
+ }
+ } catch (FileNotFoundException e) {
+ // Metadata directory does not exist
+ }
+
+ return String.format("Removed Metadata Table from %s", metadataPath);
+ }
+
+ @ShellMethod(key = "metadata init", value = "Update the metadata table from commits since the creation")
+ public String init(@ShellOption(value = "--sparkMaster", defaultValue = SparkUtil.DEFAULT_SPARK_MASTER, help = "Spark master") final String master,
+ @ShellOption(value = {"--readonly"}, defaultValue = "false",
+ help = "Open in read-only mode") final boolean readOnly) throws Exception {
+ HoodieCLI.getTableMetaClient();
+ Path metadataPath = new Path(getMetadataTableBasePath(HoodieCLI.basePath));
+ try {
+ HoodieCLI.fs.listStatus(metadataPath);
+ } catch (FileNotFoundException e) {
+ // Metadata directory does not exist
+ throw new RuntimeException("Metadata directory (" + metadataPath.toString() + ") does not exist.");
+ }
+
+ HoodieTimer timer = new HoodieTimer().startTimer();
+ if (!readOnly) {
+ HoodieWriteConfig writeConfig = getWriteConfig();
+ initJavaSparkContext(Option.of(master));
+ SparkHoodieBackedTableMetadataWriter.create(HoodieCLI.conf, writeConfig, new HoodieSparkEngineContext(jsc));
+ }
+
+ String action = readOnly ? "Opened" : "Initialized";
+ return String.format(action + " Metadata Table in %s (duration=%.2fsec)", metadataPath, (timer.endTimer()) / 1000.0);
+ }
+
+ @ShellMethod(key = "metadata stats", value = "Print stats about the metadata")
+ public String stats() throws IOException {
+ HoodieCLI.getTableMetaClient();
+ HoodieMetadataConfig config = HoodieMetadataConfig.newBuilder().enable(true).build();
+ HoodieBackedTableMetadata metadata = new HoodieBackedTableMetadata(new HoodieLocalEngineContext(HoodieCLI.conf),
+ config, HoodieCLI.basePath, "/tmp");
+ Map stats = metadata.stats();
+
+ final List rows = new ArrayList<>();
+ for (Map.Entry entry : stats.entrySet()) {
+ Comparable[] row = new Comparable[2];
+ row[0] = entry.getKey();
+ row[1] = entry.getValue();
+ rows.add(row);
+ }
+
+ TableHeader header = new TableHeader()
+ .addTableHeaderField("stat key")
+ .addTableHeaderField("stat value");
+ return HoodiePrintHelper.print(header, new HashMap<>(), "",
+ false, Integer.MAX_VALUE, false, rows);
+ }
+
+ @ShellMethod(key = "metadata list-partitions", value = "List all partitions from metadata")
+ public String listPartitions(
+ @ShellOption(value = "--sparkMaster", defaultValue = SparkUtil.DEFAULT_SPARK_MASTER, help = "Spark master") final String master
+ ) throws IOException {
+ HoodieCLI.getTableMetaClient();
+ initJavaSparkContext(Option.of(master));
+ HoodieMetadataConfig config = HoodieMetadataConfig.newBuilder().enable(true).build();
+ HoodieBackedTableMetadata metadata = new HoodieBackedTableMetadata(new HoodieSparkEngineContext(jsc), config,
+ HoodieCLI.basePath, "/tmp");
+
+ if (!metadata.enabled()) {
+ return "[ERROR] Metadata Table not enabled/initialized\n\n";
+ }
+
+ HoodieTimer timer = new HoodieTimer().startTimer();
+ List partitions = metadata.getAllPartitionPaths();
+ LOG.debug("Took " + timer.endTimer() + " ms");
+
+ final List rows = new ArrayList<>();
+ partitions.stream().sorted(Comparator.reverseOrder()).forEach(p -> {
+ Comparable[] row = new Comparable[1];
+ row[0] = p;
+ rows.add(row);
+ });
+
+ TableHeader header = new TableHeader().addTableHeaderField("partition");
+ return HoodiePrintHelper.print(header, new HashMap<>(), "",
+ false, Integer.MAX_VALUE, false, rows);
+ }
+
+ @ShellMethod(key = "metadata list-files", value = "Print a list of all files in a partition from the metadata")
+ public String listFiles(
+ @ShellOption(value = {"--partition"}, help = "Name of the partition to list files", defaultValue = "") final String partition) throws IOException {
+ HoodieCLI.getTableMetaClient();
+ HoodieMetadataConfig config = HoodieMetadataConfig.newBuilder().enable(true).build();
+ HoodieBackedTableMetadata metaReader = new HoodieBackedTableMetadata(
+ new HoodieLocalEngineContext(HoodieCLI.conf), config, HoodieCLI.basePath, "/tmp");
+
+ if (!metaReader.enabled()) {
+ return "[ERROR] Metadata Table not enabled/initialized\n\n";
+ }
+
+ Path partitionPath = new Path(HoodieCLI.basePath);
+ if (!StringUtils.isNullOrEmpty(partition)) {
+ partitionPath = new Path(HoodieCLI.basePath, partition);
+ }
+
+ HoodieTimer timer = new HoodieTimer().startTimer();
+ FileStatus[] statuses = metaReader.getAllFilesInPartition(partitionPath);
+ LOG.debug("Took " + timer.endTimer() + " ms");
+
+ final List rows = new ArrayList<>();
+ Arrays.stream(statuses).sorted((p1, p2) -> p2.getPath().getName().compareTo(p1.getPath().getName())).forEach(f -> {
+ Comparable[] row = new Comparable[1];
+ row[0] = f;
+ rows.add(row);
+ });
+
+ TableHeader header = new TableHeader().addTableHeaderField("file path");
+ return HoodiePrintHelper.print(header, new HashMap<>(), "",
+ false, Integer.MAX_VALUE, false, rows);
+ }
+
+ @ShellMethod(key = "metadata validate-files", value = "Validate all files in all partitions from the metadata")
+ public String validateFiles(
+ @ShellOption(value = {"--verbose"}, help = "Print all file details", defaultValue = "false") final boolean verbose) throws IOException {
+ HoodieCLI.getTableMetaClient();
+ HoodieMetadataConfig config = HoodieMetadataConfig.newBuilder().enable(true).build();
+ HoodieBackedTableMetadata metadataReader = new HoodieBackedTableMetadata(
+ new HoodieLocalEngineContext(HoodieCLI.conf), config, HoodieCLI.basePath, "/tmp");
+
+ if (!metadataReader.enabled()) {
+ return "[ERROR] Metadata Table not enabled/initialized\n\n";
+ }
+
+ HoodieMetadataConfig fsConfig = HoodieMetadataConfig.newBuilder().enable(false).build();
+ HoodieBackedTableMetadata fsMetaReader = new HoodieBackedTableMetadata(
+ new HoodieLocalEngineContext(HoodieCLI.conf), fsConfig, HoodieCLI.basePath, "/tmp");
+
+ HoodieTimer timer = new HoodieTimer().startTimer();
+ List metadataPartitions = metadataReader.getAllPartitionPaths();
+ LOG.debug("Listing partitions Took " + timer.endTimer() + " ms");
+ List fsPartitions = fsMetaReader.getAllPartitionPaths();
+ Collections.sort(fsPartitions);
+ Collections.sort(metadataPartitions);
+
+ Set allPartitions = new HashSet<>();
+ allPartitions.addAll(fsPartitions);
+ allPartitions.addAll(metadataPartitions);
+
+ if (!fsPartitions.equals(metadataPartitions)) {
+ LOG.error("FS partition listing is not matching with metadata partition listing!");
+ LOG.error("All FS partitions: " + Arrays.toString(fsPartitions.toArray()));
+ LOG.error("All Metadata partitions: " + Arrays.toString(metadataPartitions.toArray()));
+ }
+
+ final List rows = new ArrayList<>();
+ for (String partition : allPartitions) {
+ Map fileStatusMap = new HashMap<>();
+ Map metadataFileStatusMap = new HashMap<>();
+ FileStatus[] metadataStatuses = metadataReader.getAllFilesInPartition(new Path(HoodieCLI.basePath, partition));
+ Arrays.stream(metadataStatuses).forEach(entry -> metadataFileStatusMap.put(entry.getPath().getName(), entry));
+ FileStatus[] fsStatuses = fsMetaReader.getAllFilesInPartition(new Path(HoodieCLI.basePath, partition));
+ Arrays.stream(fsStatuses).forEach(entry -> fileStatusMap.put(entry.getPath().getName(), entry));
+
+ Set allFiles = new HashSet<>();
+ allFiles.addAll(fileStatusMap.keySet());
+ allFiles.addAll(metadataFileStatusMap.keySet());
+
+ for (String file : allFiles) {
+ Comparable[] row = new Comparable[6];
+ row[0] = partition;
+ FileStatus fsFileStatus = fileStatusMap.get(file);
+ FileStatus metaFileStatus = metadataFileStatusMap.get(file);
+ boolean doesFsFileExists = fsFileStatus != null;
+ boolean doesMetadataFileExists = metaFileStatus != null;
+ long fsFileLength = doesFsFileExists ? fsFileStatus.getLen() : 0;
+ long metadataFileLength = doesMetadataFileExists ? metaFileStatus.getLen() : 0;
+ row[1] = file;
+ row[2] = doesFsFileExists;
+ row[3] = doesMetadataFileExists;
+ row[4] = fsFileLength;
+ row[5] = metadataFileLength;
+ if (verbose) { // if verbose print all files
+ rows.add(row);
+ } else if ((doesFsFileExists != doesMetadataFileExists) || (fsFileLength != metadataFileLength)) { // if non verbose, print only non matching files
+ rows.add(row);
+ }
+ }
+
+ if (metadataStatuses.length != fsStatuses.length) {
+ LOG.error(" FS and metadata files count not matching for " + partition + ". FS files count " + fsStatuses.length + ", metadata base files count "
+ + metadataStatuses.length);
+ }
+
+ for (Map.Entry entry : fileStatusMap.entrySet()) {
+ if (!metadataFileStatusMap.containsKey(entry.getKey())) {
+ LOG.error("FS file not found in metadata " + entry.getKey());
+ } else {
+ if (entry.getValue().getLen() != metadataFileStatusMap.get(entry.getKey()).getLen()) {
+ LOG.error(" FS file size mismatch " + entry.getKey() + ", size equality "
+ + (entry.getValue().getLen() == metadataFileStatusMap.get(entry.getKey()).getLen())
+ + ". FS size " + entry.getValue().getLen() + ", metadata size "
+ + metadataFileStatusMap.get(entry.getKey()).getLen());
+ }
+ }
+ }
+ for (Map.Entry entry : metadataFileStatusMap.entrySet()) {
+ if (!fileStatusMap.containsKey(entry.getKey())) {
+ LOG.error("Metadata file not found in FS " + entry.getKey());
+ } else {
+ if (entry.getValue().getLen() != fileStatusMap.get(entry.getKey()).getLen()) {
+ LOG.error(" Metadata file size mismatch " + entry.getKey() + ", size equality "
+ + (entry.getValue().getLen() == fileStatusMap.get(entry.getKey()).getLen())
+ + ". Metadata size " + entry.getValue().getLen() + ", FS size "
+ + metadataFileStatusMap.get(entry.getKey()).getLen());
+ }
+ }
+ }
+ }
+ TableHeader header = new TableHeader().addTableHeaderField("Partition")
+ .addTableHeaderField("File Name")
+ .addTableHeaderField(" Is Present in FS ")
+ .addTableHeaderField(" Is Present in Metadata")
+ .addTableHeaderField(" FS size")
+ .addTableHeaderField(" Metadata size");
+ return HoodiePrintHelper.print(header, new HashMap<>(), "", false, Integer.MAX_VALUE, false, rows);
+ }
+
+ private HoodieWriteConfig getWriteConfig() {
+ return HoodieWriteConfig.newBuilder().withPath(HoodieCLI.basePath)
+ .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(true).build()).build();
+ }
+
+ private void initJavaSparkContext(Option userDefinedMaster) {
+ if (jsc == null) {
+ jsc = SparkUtil.initJavaSparkContext(SparkUtil.getDefaultConf("HoodieCLI", userDefinedMaster));
+ }
+ }
+}
\ No newline at end of file
diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/RepairsCommand.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/RepairsCommand.java
index 40dddfc725488..2b11e20a10d42 100644
--- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/RepairsCommand.java
+++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/RepairsCommand.java
@@ -18,33 +18,35 @@
package org.apache.hudi.cli.commands;
+import org.apache.avro.AvroRuntimeException;
+import org.apache.hadoop.fs.Path;
import org.apache.hudi.cli.DeDupeType;
import org.apache.hudi.cli.HoodieCLI;
import org.apache.hudi.cli.HoodiePrintHelper;
import org.apache.hudi.cli.HoodieTableHeaderFields;
import org.apache.hudi.cli.utils.InputStreamConsumer;
import org.apache.hudi.cli.utils.SparkUtil;
+import org.apache.hudi.common.engine.HoodieLocalEngineContext;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.HoodiePartitionMetadata;
import org.apache.hudi.common.table.HoodieTableConfig;
import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.common.util.CleanerUtils;
-import org.apache.hudi.exception.HoodieIOException;
-
-import org.apache.avro.AvroRuntimeException;
-import org.apache.hadoop.fs.Path;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.PartitionPathEncodeUtils;
import org.apache.hudi.common.util.StringUtils;
-import org.apache.log4j.Logger;
+import org.apache.hudi.exception.HoodieIOException;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
import org.apache.spark.launcher.SparkLauncher;
import org.apache.spark.util.Utils;
-import org.springframework.shell.core.CommandMarker;
-import org.springframework.shell.core.annotation.CliCommand;
-import org.springframework.shell.core.annotation.CliOption;
-import org.springframework.stereotype.Component;
+import org.springframework.shell.standard.ShellComponent;
+import org.springframework.shell.standard.ShellMethod;
+import org.springframework.shell.standard.ShellOption;
import scala.collection.JavaConverters;
-import java.io.File;
import java.io.FileInputStream;
import java.io.IOException;
import java.util.List;
@@ -58,29 +60,29 @@
/**
* CLI command to display and trigger repair options.
*/
-@Component
-public class RepairsCommand implements CommandMarker {
+@ShellComponent
+public class RepairsCommand {
- private static final Logger LOG = Logger.getLogger(RepairsCommand.class);
+ private static final Logger LOG = LogManager.getLogger(RepairsCommand.class);
public static final String DEDUPLICATE_RETURN_PREFIX = "Deduplicated files placed in: ";
- @CliCommand(value = "repair deduplicate",
- help = "De-duplicate a partition path contains duplicates & produce repaired files to replace with")
+ @ShellMethod(key = "repair deduplicate",
+ value = "De-duplicate a partition path contains duplicates & produce repaired files to replace with")
public String deduplicate(
- @CliOption(key = {"duplicatedPartitionPath"}, help = "Partition Path containing the duplicates",
- mandatory = true) final String duplicatedPartitionPath,
- @CliOption(key = {"repairedOutputPath"}, help = "Location to place the repaired files",
- mandatory = true) final String repairedOutputPath,
- @CliOption(key = {"sparkProperties"}, help = "Spark Properties File Path",
- unspecifiedDefaultValue = "") String sparkPropertiesPath,
- @CliOption(key = "sparkMaster", unspecifiedDefaultValue = "", help = "Spark Master") String master,
- @CliOption(key = "sparkMemory", unspecifiedDefaultValue = "4G",
+ @ShellOption(value = {"--duplicatedPartitionPath"}, defaultValue = "", help = "Partition Path containing the duplicates")
+ final String duplicatedPartitionPath,
+ @ShellOption(value = {"--repairedOutputPath"}, help = "Location to place the repaired files")
+ final String repairedOutputPath,
+ @ShellOption(value = {"--sparkProperties"}, help = "Spark Properties File Path",
+ defaultValue = "") String sparkPropertiesPath,
+ @ShellOption(value = "--sparkMaster", defaultValue = "", help = "Spark Master") String master,
+ @ShellOption(value = "--sparkMemory", defaultValue = "4G",
help = "Spark executor memory") final String sparkMemory,
- @CliOption(key = {"dryrun"},
+ @ShellOption(value = {"--dryrun"},
help = "Should we actually remove duplicates or just run and store result to repairedOutputPath",
- unspecifiedDefaultValue = "true") final boolean dryRun,
- @CliOption(key = {"dedupeType"}, help = "Valid values are - insert_type, update_type and upsert_type",
- unspecifiedDefaultValue = "insert_type") final String dedupeType)
+ defaultValue = "true") final boolean dryRun,
+ @ShellOption(value = {"--dedupeType"}, help = "Valid values are - insert_type, update_type and upsert_type",
+ defaultValue = "insert_type") final String dedupeType)
throws Exception {
if (!DeDupeType.values().contains(DeDupeType.withName(dedupeType))) {
throw new IllegalArgumentException("Please provide valid dedupe type!");
@@ -108,10 +110,10 @@ public String deduplicate(
}
}
- @CliCommand(value = "repair addpartitionmeta", help = "Add partition metadata to a table, if not present")
+ @ShellMethod(key = "repair addpartitionmeta", value = "Add partition metadata to a table, if not present")
public String addPartitionMeta(
- @CliOption(key = {"dryrun"}, help = "Should we actually add or just print what would be done",
- unspecifiedDefaultValue = "true") final boolean dryRun)
+ @ShellOption(value = {"--dryrun"}, help = "Should we actually add or just print what would be done",
+ defaultValue = "true") final boolean dryRun)
throws IOException {
HoodieTableMetaClient client = HoodieCLI.getTableMetaClient();
@@ -133,7 +135,8 @@ public String addPartitionMeta(
row[1] = "No";
if (!dryRun) {
HoodiePartitionMetadata partitionMetadata =
- new HoodiePartitionMetadata(HoodieCLI.fs, latestCommit, basePath, partitionPath);
+ new HoodiePartitionMetadata(HoodieCLI.fs, latestCommit, basePath, partitionPath,
+ client.getTableConfig().getPartitionMetafileFormat());
partitionMetadata.trySave(0);
row[2] = "Repaired";
}
@@ -145,17 +148,21 @@ public String addPartitionMeta(
HoodieTableHeaderFields.HEADER_METADATA_PRESENT, HoodieTableHeaderFields.HEADER_ACTION}, rows);
}
- @CliCommand(value = "repair overwrite-hoodie-props", help = "Overwrite hoodie.properties with provided file. Risky operation. Proceed with caution!")
+ @ShellMethod(key = "repair overwrite-hoodie-props",
+ value = "Overwrite hoodie.properties with provided file. Risky operation. Proceed with caution!")
public String overwriteHoodieProperties(
- @CliOption(key = {"new-props-file"}, help = "Path to a properties file on local filesystem to overwrite the table's hoodie.properties with")
+ @ShellOption(value = {"--new-props-file"},
+ help = "Path to a properties file on local filesystem to overwrite the table's hoodie.properties with")
final String overwriteFilePath) throws IOException {
HoodieTableMetaClient client = HoodieCLI.getTableMetaClient();
Properties newProps = new Properties();
- newProps.load(new FileInputStream(new File(overwriteFilePath)));
- Map oldProps = client.getTableConfig().getProps();
+ newProps.load(new FileInputStream(overwriteFilePath));
+ Map oldProps = client.getTableConfig().propsMap();
Path metaPathDir = new Path(client.getBasePath(), METAFOLDER_NAME);
- HoodieTableConfig.createHoodieProperties(client.getFs(), metaPathDir, newProps);
+ HoodieTableConfig.create(client.getFs(), metaPathDir, newProps);
+ // reload new props as checksum would have been added
+ newProps = HoodieTableMetaClient.reload(HoodieCLI.getTableMetaClient()).getTableConfig().getProps();
TreeSet allPropKeys = new TreeSet<>();
allPropKeys.addAll(newProps.keySet().stream().map(Object::toString).collect(Collectors.toSet()));
@@ -164,7 +171,7 @@ public String overwriteHoodieProperties(
String[][] rows = new String[allPropKeys.size()][];
int ind = 0;
for (String propKey : allPropKeys) {
- String[] row = new String[]{
+ String[] row = new String[] {
propKey,
oldProps.getOrDefault(propKey, "null"),
newProps.getOrDefault(propKey, "null").toString()
@@ -175,7 +182,7 @@ public String overwriteHoodieProperties(
HoodieTableHeaderFields.HEADER_OLD_VALUE, HoodieTableHeaderFields.HEADER_NEW_VALUE}, rows);
}
- @CliCommand(value = "repair corrupted clean files", help = "repair corrupted clean files")
+ @ShellMethod(key = "repair corrupted clean files", value = "repair corrupted clean files")
public void removeCorruptedPendingCleanAction() {
HoodieTableMetaClient client = HoodieCLI.getTableMetaClient();
@@ -186,15 +193,130 @@ public void removeCorruptedPendingCleanAction() {
CleanerUtils.getCleanerPlan(client, instant);
} catch (AvroRuntimeException e) {
LOG.warn("Corruption found. Trying to remove corrupted clean instant file: " + instant);
- FSUtils.deleteInstantFile(client.getFs(), client.getMetaPath(), instant);
+ HoodieActiveTimeline.deleteInstantFile(client.getFs(), client.getMetaPath(), instant);
} catch (IOException ioe) {
if (ioe.getMessage().contains("Not an Avro data file")) {
LOG.warn("Corruption found. Trying to remove corrupted clean instant file: " + instant);
- FSUtils.deleteInstantFile(client.getFs(), client.getMetaPath(), instant);
+ HoodieActiveTimeline.deleteInstantFile(client.getFs(), client.getMetaPath(), instant);
} else {
throw new HoodieIOException(ioe.getMessage(), ioe);
}
}
});
}
+
+ @ShellMethod(key = "repair migrate-partition-meta", value = "Migrate all partition meta file currently stored in text format "
+ + "to be stored in base file format. See HoodieTableConfig#PARTITION_METAFILE_USE_DATA_FORMAT.")
+ public String migratePartitionMeta(
+ @ShellOption(value = {"--dryrun"}, help = "dry run without modifying anything.", defaultValue = "true")
+ final boolean dryRun)
+ throws IOException {
+
+ HoodieLocalEngineContext engineContext = new HoodieLocalEngineContext(HoodieCLI.conf);
+ HoodieTableMetaClient client = HoodieCLI.getTableMetaClient();
+ List partitionPaths = FSUtils.getAllPartitionPaths(engineContext, client.getBasePath(), false, false);
+ Path basePath = new Path(client.getBasePath());
+
+ String[][] rows = new String[partitionPaths.size()][];
+ int ind = 0;
+ for (String partitionPath : partitionPaths) {
+ Path partition = FSUtils.getPartitionPath(client.getBasePath(), partitionPath);
+ Option textFormatFile = HoodiePartitionMetadata.textFormatMetaPathIfExists(HoodieCLI.fs, partition);
+ Option baseFormatFile = HoodiePartitionMetadata.baseFormatMetaPathIfExists(HoodieCLI.fs, partition);
+ String latestCommit = client.getActiveTimeline().getCommitTimeline().lastInstant().get().getTimestamp();
+
+ String[] row = new String[] {
+ partitionPath,
+ String.valueOf(textFormatFile.isPresent()),
+ String.valueOf(baseFormatFile.isPresent()),
+ textFormatFile.isPresent() ? "MIGRATE" : "NONE"
+ };
+
+ if (!dryRun) {
+ if (!baseFormatFile.isPresent()) {
+ HoodiePartitionMetadata partitionMetadata = new HoodiePartitionMetadata(HoodieCLI.fs, latestCommit, basePath, partition,
+ Option.of(client.getTableConfig().getBaseFileFormat()));
+ partitionMetadata.trySave(0);
+ }
+
+ // delete it, in case we failed midway last time.
+ textFormatFile.ifPresent(path -> {
+ try {
+ HoodieCLI.fs.delete(path, false);
+ } catch (IOException e) {
+ throw new HoodieIOException(e.getMessage(), e);
+ }
+ });
+
+ row[3] = "MIGRATED";
+ }
+
+ rows[ind++] = row;
+ }
+
+ Properties props = new Properties();
+ props.setProperty(HoodieTableConfig.PARTITION_METAFILE_USE_BASE_FORMAT.key(), "true");
+ HoodieTableConfig.update(HoodieCLI.fs, new Path(client.getMetaPath()), props);
+
+ return HoodiePrintHelper.print(new String[] {
+ HoodieTableHeaderFields.HEADER_PARTITION_PATH,
+ HoodieTableHeaderFields.HEADER_TEXT_METAFILE_PRESENT,
+ HoodieTableHeaderFields.HEADER_BASE_METAFILE_PRESENT,
+ HoodieTableHeaderFields.HEADER_ACTION
+ }, rows);
+ }
+
+ @ShellMethod(key = "repair deprecated partition",
+ value = "Repair deprecated partition (\"default\"). Re-writes data from the deprecated partition into " + PartitionPathEncodeUtils.DEFAULT_PARTITION_PATH)
+ public String repairDeprecatePartition(
+ @ShellOption(value = {"--sparkProperties"}, help = "Spark Properties File Path",
+ defaultValue = "") String sparkPropertiesPath,
+ @ShellOption(value = "--sparkMaster", defaultValue = "", help = "Spark Master") String master,
+ @ShellOption(value = "--sparkMemory", defaultValue = "4G",
+ help = "Spark executor memory") final String sparkMemory) throws Exception {
+ if (StringUtils.isNullOrEmpty(sparkPropertiesPath)) {
+ sparkPropertiesPath =
+ Utils.getDefaultPropertiesFile(JavaConverters.mapAsScalaMapConverter(System.getenv()).asScala());
+ }
+
+ SparkLauncher sparkLauncher = SparkUtil.initLauncher(sparkPropertiesPath);
+ sparkLauncher.addAppArgs(SparkMain.SparkCommand.REPAIR_DEPRECATED_PARTITION.toString(), master, sparkMemory,
+ HoodieCLI.getTableMetaClient().getBasePathV2().toString());
+ Process process = sparkLauncher.launch();
+ InputStreamConsumer.captureOutput(process);
+ int exitCode = process.waitFor();
+
+ if (exitCode != 0) {
+ return "Deduplication failed!";
+ }
+ return "Repair succeeded";
+ }
+
+ @ShellMethod(key = "rename partition",
+ value = "Rename partition. Usage: rename partition --oldPartition --newPartition ")
+ public String renamePartition(
+ @ShellOption(value = {"--oldPartition"}, help = "Partition value to be renamed") String oldPartition,
+ @ShellOption(value = {"--newPartition"}, help = "New partition value after rename") String newPartition,
+ @ShellOption(value = {"--sparkProperties"}, help = "Spark Properties File Path",
+ defaultValue = "") String sparkPropertiesPath,
+ @ShellOption(value = "--sparkMaster", defaultValue = "", help = "Spark Master") String master,
+ @ShellOption(value = "--sparkMemory", defaultValue = "4G",
+ help = "Spark executor memory") final String sparkMemory) throws Exception {
+ if (StringUtils.isNullOrEmpty(sparkPropertiesPath)) {
+ sparkPropertiesPath =
+ Utils.getDefaultPropertiesFile(JavaConverters.mapAsScalaMapConverter(System.getenv()).asScala());
+ }
+
+ SparkLauncher sparkLauncher = SparkUtil.initLauncher(sparkPropertiesPath);
+ sparkLauncher.addAppArgs(SparkMain.SparkCommand.RENAME_PARTITION.toString(), master, sparkMemory,
+ HoodieCLI.getTableMetaClient().getBasePathV2().toString(), oldPartition, newPartition);
+ Process process = sparkLauncher.launch();
+ InputStreamConsumer.captureOutput(process);
+ int exitCode = process.waitFor();
+
+ if (exitCode != 0) {
+ return "rename partition failed!";
+ }
+ return "rename partition succeeded";
+ }
}
diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/RollbacksCommand.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/RollbacksCommand.java
index faa778943c9b0..e0fad70d99b3e 100644
--- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/RollbacksCommand.java
+++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/RollbacksCommand.java
@@ -23,6 +23,8 @@
import org.apache.hudi.cli.HoodiePrintHelper;
import org.apache.hudi.cli.HoodieTableHeaderFields;
import org.apache.hudi.cli.TableHeader;
+import org.apache.hudi.cli.utils.InputStreamConsumer;
+import org.apache.hudi.cli.utils.SparkUtil;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
import org.apache.hudi.common.table.timeline.HoodieInstant;
@@ -31,11 +33,10 @@
import org.apache.hudi.common.table.timeline.TimelineMetadataUtils;
import org.apache.hudi.common.util.CollectionUtils;
import org.apache.hudi.common.util.collection.Pair;
-
-import org.springframework.shell.core.CommandMarker;
-import org.springframework.shell.core.annotation.CliCommand;
-import org.springframework.shell.core.annotation.CliOption;
-import org.springframework.stereotype.Component;
+import org.apache.spark.launcher.SparkLauncher;
+import org.springframework.shell.standard.ShellComponent;
+import org.springframework.shell.standard.ShellMethod;
+import org.springframework.shell.standard.ShellOption;
import java.io.IOException;
import java.util.ArrayList;
@@ -48,16 +49,16 @@
/**
* CLI command to display rollback options.
*/
-@Component
-public class RollbacksCommand implements CommandMarker {
+@ShellComponent
+public class RollbacksCommand {
- @CliCommand(value = "show rollbacks", help = "List all rollback instants")
+ @ShellMethod(key = "show rollbacks", value = "List all rollback instants")
public String showRollbacks(
- @CliOption(key = {"limit"}, help = "Limit #rows to be displayed", unspecifiedDefaultValue = "10") Integer limit,
- @CliOption(key = {"sortBy"}, help = "Sorting Field", unspecifiedDefaultValue = "") final String sortByField,
- @CliOption(key = {"desc"}, help = "Ordering", unspecifiedDefaultValue = "false") final boolean descending,
- @CliOption(key = {"headeronly"}, help = "Print Header Only",
- unspecifiedDefaultValue = "false") final boolean headerOnly) {
+ @ShellOption(value = {"--limit"}, help = "Limit #rows to be displayed", defaultValue = "10") Integer limit,
+ @ShellOption(value = {"--sortBy"}, help = "Sorting Field", defaultValue = "") final String sortByField,
+ @ShellOption(value = {"--desc"}, help = "Ordering", defaultValue = "false") final boolean descending,
+ @ShellOption(value = {"--headeronly"}, help = "Print Header Only",
+ defaultValue = "false") final boolean headerOnly) {
HoodieActiveTimeline activeTimeline = new RollbackTimeline(HoodieCLI.getTableMetaClient());
HoodieTimeline rollback = activeTimeline.getRollbackTimeline().filterCompletedInstants();
@@ -87,14 +88,14 @@ public String showRollbacks(
return HoodiePrintHelper.print(header, new HashMap<>(), sortByField, descending, limit, headerOnly, rows);
}
- @CliCommand(value = "show rollback", help = "Show details of a rollback instant")
+ @ShellMethod(key = "show rollback", value = "Show details of a rollback instant")
public String showRollback(
- @CliOption(key = {"instant"}, help = "Rollback instant", mandatory = true) String rollbackInstant,
- @CliOption(key = {"limit"}, help = "Limit #rows to be displayed", unspecifiedDefaultValue = "10") Integer limit,
- @CliOption(key = {"sortBy"}, help = "Sorting Field", unspecifiedDefaultValue = "") final String sortByField,
- @CliOption(key = {"desc"}, help = "Ordering", unspecifiedDefaultValue = "false") final boolean descending,
- @CliOption(key = {"headeronly"}, help = "Print Header Only",
- unspecifiedDefaultValue = "false") final boolean headerOnly)
+ @ShellOption(value = {"--instant"}, help = "Rollback instant") String rollbackInstant,
+ @ShellOption(value = {"--limit"}, help = "Limit #rows to be displayed", defaultValue = "10") Integer limit,
+ @ShellOption(value = {"--sortBy"}, help = "Sorting Field", defaultValue = "") final String sortByField,
+ @ShellOption(value = {"--desc"}, help = "Ordering", defaultValue = "false") final boolean descending,
+ @ShellOption(value = {"--headeronly"}, help = "Print Header Only",
+ defaultValue = "false") final boolean headerOnly)
throws IOException {
HoodieActiveTimeline activeTimeline = new RollbackTimeline(HoodieCLI.getTableMetaClient());
final List rows = new ArrayList<>();
@@ -102,17 +103,17 @@ public String showRollback(
activeTimeline.getInstantDetails(new HoodieInstant(State.COMPLETED, ROLLBACK_ACTION, rollbackInstant)).get(),
HoodieRollbackMetadata.class);
metadata.getPartitionMetadata().forEach((key, value) -> Stream
- .concat(value.getSuccessDeleteFiles().stream().map(f -> Pair.of(f, true)),
- value.getFailedDeleteFiles().stream().map(f -> Pair.of(f, false)))
- .forEach(fileWithDeleteStatus -> {
- Comparable[] row = new Comparable[5];
- row[0] = metadata.getStartRollbackTime();
- row[1] = metadata.getCommitsRollback().toString();
- row[2] = key;
- row[3] = fileWithDeleteStatus.getLeft();
- row[4] = fileWithDeleteStatus.getRight();
- rows.add(row);
- }));
+ .concat(value.getSuccessDeleteFiles().stream().map(f -> Pair.of(f, true)),
+ value.getFailedDeleteFiles().stream().map(f -> Pair.of(f, false)))
+ .forEach(fileWithDeleteStatus -> {
+ Comparable[] row = new Comparable[5];
+ row[0] = metadata.getStartRollbackTime();
+ row[1] = metadata.getCommitsRollback().toString();
+ row[2] = key;
+ row[3] = fileWithDeleteStatus.getLeft();
+ row[4] = fileWithDeleteStatus.getRight();
+ rows.add(row);
+ }));
TableHeader header = new TableHeader().addTableHeaderField(HoodieTableHeaderFields.HEADER_INSTANT)
.addTableHeaderField(HoodieTableHeaderFields.HEADER_ROLLBACK_INSTANT)
@@ -122,6 +123,38 @@ public String showRollback(
return HoodiePrintHelper.print(header, new HashMap<>(), sortByField, descending, limit, headerOnly, rows);
}
+ @ShellMethod(key = "commit rollback", value = "Rollback a commit")
+ public String rollbackCommit(
+ @ShellOption(value = {"--commit"}, help = "Commit to rollback") final String instantTime,
+ @ShellOption(value = {"--sparkProperties"}, help = "Spark Properties File Path",
+ defaultValue = "") final String sparkPropertiesPath,
+ @ShellOption(value = "--sparkMaster", defaultValue = "", help = "Spark Master") String master,
+ @ShellOption(value = "--sparkMemory", defaultValue = "4G",
+ help = "Spark executor memory") final String sparkMemory,
+ @ShellOption(value = "--rollbackUsingMarkers", defaultValue = "false",
+ help = "Enabling marker based rollback") final String rollbackUsingMarkers)
+ throws Exception {
+ HoodieActiveTimeline activeTimeline = HoodieCLI.getTableMetaClient().getActiveTimeline();
+ HoodieTimeline completedTimeline = activeTimeline.getCommitsTimeline().filterCompletedInstants();
+ HoodieTimeline filteredTimeline = completedTimeline.filter(instant -> instant.getTimestamp().equals(instantTime));
+ if (filteredTimeline.empty()) {
+ return "Commit " + instantTime + " not found in Commits " + completedTimeline;
+ }
+
+ SparkLauncher sparkLauncher = SparkUtil.initLauncher(sparkPropertiesPath);
+ sparkLauncher.addAppArgs(SparkMain.SparkCommand.ROLLBACK.toString(), master, sparkMemory, instantTime,
+ HoodieCLI.getTableMetaClient().getBasePath(), rollbackUsingMarkers);
+ Process process = sparkLauncher.launch();
+ InputStreamConsumer.captureOutput(process);
+ int exitCode = process.waitFor();
+ // Refresh the current
+ HoodieCLI.refreshTableMetadata();
+ if (exitCode != 0) {
+ return "Commit " + instantTime + " failed to roll back";
+ }
+ return "Commit " + instantTime + " rolled back";
+ }
+
/**
* An Active timeline containing only rollbacks.
*/
diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SavepointsCommand.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SavepointsCommand.java
index e4d7cc69e9eb1..73f94acda8787 100644
--- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SavepointsCommand.java
+++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SavepointsCommand.java
@@ -23,23 +23,15 @@
import org.apache.hudi.cli.HoodieTableHeaderFields;
import org.apache.hudi.cli.utils.InputStreamConsumer;
import org.apache.hudi.cli.utils.SparkUtil;
-import org.apache.hudi.client.SparkRDDWriteClient;
-import org.apache.hudi.client.common.HoodieSparkEngineContext;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
-import org.apache.hudi.config.HoodieIndexConfig;
-import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieException;
-import org.apache.hudi.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.CliCommand;
-import org.springframework.shell.core.annotation.CliOption;
-import org.springframework.stereotype.Component;
+import org.springframework.shell.standard.ShellComponent;
+import org.springframework.shell.standard.ShellMethod;
+import org.springframework.shell.standard.ShellOption;
import java.util.List;
import java.util.stream.Collectors;
@@ -47,10 +39,10 @@
/**
* CLI command to display savepoint options.
*/
-@Component
-public class SavepointsCommand implements CommandMarker {
+@ShellComponent
+public class SavepointsCommand {
- @CliCommand(value = "savepoints show", help = "Show the savepoints")
+ @ShellMethod(key = "savepoints show", value = "Show the savepoints")
public String showSavepoints() {
HoodieActiveTimeline activeTimeline = HoodieCLI.getTableMetaClient().getActiveTimeline();
HoodieTimeline timeline = activeTimeline.getSavePointTimeline().filterCompletedInstants();
@@ -63,24 +55,24 @@ public String showSavepoints() {
return HoodiePrintHelper.print(new String[] {HoodieTableHeaderFields.HEADER_SAVEPOINT_TIME}, 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"}, unspecifiedDefaultValue = "default",
+ @ShellMethod(key = "savepoint create", value = "Savepoint a commit")
+ public String savepoint(
+ @ShellOption(value = {"--commit"}, help = "Commit to savepoint") final String commitTime,
+ @ShellOption(value = {"--user"}, defaultValue = "default",
help = "User who is creating the savepoint") final String user,
- @CliOption(key = {"comments"}, unspecifiedDefaultValue = "default",
+ @ShellOption(value = {"--comments"}, defaultValue = "default",
help = "Comments for creating the savepoint") final String comments,
- @CliOption(key = {"sparkProperties"}, help = "Spark Properties File Path") final String sparkPropertiesPath,
- @CliOption(key = "sparkMaster", unspecifiedDefaultValue = "", help = "Spark Master") String master,
- @CliOption(key = "sparkMemory", unspecifiedDefaultValue = "4G",
+ @ShellOption(value = {"--sparkProperties"}, help = "Spark Properties File Path",
+ defaultValue = "") final String sparkPropertiesPath,
+ @ShellOption(value = "--sparkMaster", defaultValue = "", help = "Spark Master") String master,
+ @ShellOption(value = "--sparkMemory", defaultValue = "4G",
help = "Spark executor memory") final String sparkMemory)
throws Exception {
HoodieTableMetaClient metaClient = HoodieCLI.getTableMetaClient();
HoodieActiveTimeline activeTimeline = metaClient.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;
+ if (!activeTimeline.getCommitsTimeline().filterCompletedInstants().containsInstant(commitTime)) {
+ return "Commit " + commitTime + " not found in Commits " + activeTimeline;
}
SparkLauncher sparkLauncher = SparkUtil.initLauncher(sparkPropertiesPath);
@@ -97,12 +89,15 @@ public String savepoint(@CliOption(key = {"commit"}, help = "Commit to savepoint
return String.format("The commit \"%s\" has been savepointed.", commitTime);
}
- @CliCommand(value = "savepoint rollback", help = "Savepoint a commit")
+ @ShellMethod(key = "savepoint rollback", value = "Savepoint a commit")
public String rollbackToSavepoint(
- @CliOption(key = {"savepoint"}, help = "Savepoint to rollback") final String instantTime,
- @CliOption(key = {"sparkProperties"}, help = "Spark Properties File Path") final String sparkPropertiesPath,
- @CliOption(key = "sparkMaster", unspecifiedDefaultValue = "", help = "Spark Master") String master,
- @CliOption(key = "sparkMemory", unspecifiedDefaultValue = "4G",
+ @ShellOption(value = {"--savepoint"}, help = "Savepoint to rollback") final String instantTime,
+ @ShellOption(value = {"--sparkProperties"}, help = "Spark Properties File Path",
+ defaultValue = "") final String sparkPropertiesPath,
+ @ShellOption(value = "--sparkMaster", defaultValue = "", help = "Spark Master") String master,
+ @ShellOption(value = {"--lazyFailedWritesCleanPolicy"}, help = "True if FailedWriteCleanPolicy is lazy",
+ defaultValue = "false") final String lazyFailedWritesCleanPolicy,
+ @ShellOption(value = "--sparkMemory", defaultValue = "4G",
help = "Spark executor memory") final String sparkMemory)
throws Exception {
HoodieTableMetaClient metaClient = HoodieCLI.getTableMetaClient();
@@ -110,16 +105,16 @@ public String rollbackToSavepoint(
throw new HoodieException("There are no completed instants to run rollback");
}
HoodieActiveTimeline activeTimeline = metaClient.getActiveTimeline();
- HoodieTimeline timeline = activeTimeline.getCommitTimeline().filterCompletedInstants();
- HoodieInstant commitInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, instantTime);
+ HoodieTimeline timeline = activeTimeline.getCommitsTimeline().filterCompletedInstants();
+ List instants = timeline.getInstants().filter(instant -> instant.getTimestamp().equals(instantTime)).collect(Collectors.toList());
- if (!timeline.containsInstant(commitInstant)) {
+ if (instants.isEmpty()) {
return "Commit " + instantTime + " not found in Commits " + timeline;
}
SparkLauncher sparkLauncher = SparkUtil.initLauncher(sparkPropertiesPath);
sparkLauncher.addAppArgs(SparkMain.SparkCommand.ROLLBACK_TO_SAVEPOINT.toString(), master, sparkMemory,
- instantTime, metaClient.getBasePath());
+ instantTime, metaClient.getBasePath(), lazyFailedWritesCleanPolicy);
Process process = sparkLauncher.launch();
InputStreamConsumer.captureOutput(process);
int exitCode = process.waitFor();
@@ -131,11 +126,13 @@ public String rollbackToSavepoint(
return String.format("Savepoint \"%s\" rolled back", instantTime);
}
- @CliCommand(value = "savepoint delete", help = "Delete the savepoint")
- public String deleteSavepoint(@CliOption(key = {"commit"}, help = "Delete a savepoint") final String instantTime,
- @CliOption(key = {"sparkProperties"}, help = "Spark Properties File Path") final String sparkPropertiesPath,
- @CliOption(key = "sparkMaster", unspecifiedDefaultValue = "", help = "Spark Master") String master,
- @CliOption(key = "sparkMemory", unspecifiedDefaultValue = "4G",
+ @ShellMethod(key = "savepoint delete", value = "Delete the savepoint")
+ public String deleteSavepoint(
+ @ShellOption(value = {"--commit"}, help = "Delete a savepoint") final String instantTime,
+ @ShellOption(value = {"--sparkProperties"}, help = "Spark Properties File Path",
+ defaultValue = "") final String sparkPropertiesPath,
+ @ShellOption(value = "--sparkMaster", defaultValue = "", help = "Spark Master") String master,
+ @ShellOption(value = "--sparkMemory", defaultValue = "4G",
help = "Spark executor memory") final String sparkMemory)
throws Exception {
HoodieTableMetaClient metaClient = HoodieCLI.getTableMetaClient();
@@ -162,11 +159,4 @@ public String deleteSavepoint(@CliOption(key = {"commit"}, help = "Delete a save
}
return String.format("Savepoint \"%s\" deleted.", instantTime);
}
-
- private static SparkRDDWriteClient createHoodieClient(JavaSparkContext jsc, String basePath) throws Exception {
- HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath)
- .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()).build();
- return new SparkRDDWriteClient(new HoodieSparkEngineContext(jsc), config, false);
- }
-
}
diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SparkEnvCommand.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SparkEnvCommand.java
index 7969808e29831..02778ac2cff60 100644
--- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SparkEnvCommand.java
+++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SparkEnvCommand.java
@@ -19,11 +19,9 @@
package org.apache.hudi.cli.commands;
import org.apache.hudi.cli.HoodiePrintHelper;
-
-import org.springframework.shell.core.CommandMarker;
-import org.springframework.shell.core.annotation.CliCommand;
-import org.springframework.shell.core.annotation.CliOption;
-import org.springframework.stereotype.Component;
+import org.springframework.shell.standard.ShellComponent;
+import org.springframework.shell.standard.ShellMethod;
+import org.springframework.shell.standard.ShellOption;
import java.util.HashMap;
import java.util.Map;
@@ -31,37 +29,38 @@
/**
* CLI command to set and show spark launcher init env.
*/
-@Component
-public class SparkEnvCommand implements CommandMarker {
+@ShellComponent
+public class SparkEnvCommand {
public static Map env = new HashMap<>();
- @CliCommand(value = "set", help = "Set spark launcher env to cli")
- public void setEnv(@CliOption(key = {"conf"}, help = "Env config to be set") final String confMap) {
+ @ShellMethod(key = "set", value = "Set spark launcher env to cli")
+ public void setEnv(@ShellOption(value = {"--conf"}, help = "Env config to be set") final String confMap) {
String[] map = confMap.split("=");
if (map.length != 2) {
throw new IllegalArgumentException("Illegal set parameter, please use like [set --conf SPARK_HOME=/usr/etc/spark]");
}
env.put(map[0].trim(), map[1].trim());
+ System.setProperty(map[0].trim(), map[1].trim());
}
- @CliCommand(value = "show envs all", help = "Show spark launcher envs")
+ @ShellMethod(key = "show envs all", value = "Show spark launcher envs")
public String showAllEnv() {
String[][] rows = new String[env.size()][2];
int i = 0;
- for (Map.Entry entry: env.entrySet()) {
- rows[i] = new String[]{entry.getKey(), entry.getValue()};
+ for (Map.Entry entry : env.entrySet()) {
+ rows[i] = new String[] {entry.getKey(), entry.getValue()};
i++;
}
return HoodiePrintHelper.print(new String[] {"key", "value"}, rows);
}
- @CliCommand(value = "show env", help = "Show spark launcher env by key")
- public String showEnvByKey(@CliOption(key = {"key"}, help = "Which env conf want to show") final String key) {
+ @ShellMethod(key = "show env", value = "Show spark launcher env by key")
+ public String showEnvByKey(@ShellOption(value = {"--key"}, help = "Which env conf want to show") final String key) {
if (key == null || key.isEmpty()) {
return showAllEnv();
} else {
- return HoodiePrintHelper.print(new String[] {"key", "value"}, new String[][]{new String[]{key, env.get(key)}});
+ return HoodiePrintHelper.print(new String[] {"key", "value"}, new String[][] {new String[] {key, env.getOrDefault(key, "")}});
}
}
}
diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SparkMain.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SparkMain.java
index f715b16e012c3..4abfe48e1119e 100644
--- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SparkMain.java
+++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SparkMain.java
@@ -18,200 +18,292 @@
package org.apache.hudi.cli.commands;
-import org.apache.hudi.cli.DeDupeType;
import org.apache.hudi.DataSourceWriteOptions;
+import org.apache.hudi.cli.DeDupeType;
import org.apache.hudi.cli.DedupeSparkJob;
import org.apache.hudi.cli.utils.SparkUtil;
import org.apache.hudi.client.SparkRDDWriteClient;
import org.apache.hudi.client.common.HoodieSparkEngineContext;
import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.engine.HoodieEngineContext;
import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.WriteOperationType;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.HoodieTableVersion;
import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion;
import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.PartitionPathEncodeUtils;
import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.common.util.ValidationUtils;
import org.apache.hudi.config.HoodieBootstrapConfig;
+import org.apache.hudi.config.HoodieCleanConfig;
import org.apache.hudi.config.HoodieIndexConfig;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieSavepointException;
import org.apache.hudi.index.HoodieIndex;
+import org.apache.hudi.keygen.constant.KeyGeneratorType;
+import org.apache.hudi.table.HoodieSparkTable;
import org.apache.hudi.table.action.compact.strategy.UnBoundedCompactionStrategy;
-import org.apache.hudi.table.upgrade.SparkUpgradeDowngrade;
+import org.apache.hudi.table.marker.WriteMarkersFactory;
+import org.apache.hudi.table.upgrade.SparkUpgradeDowngradeHelper;
+import org.apache.hudi.table.upgrade.UpgradeDowngrade;
import org.apache.hudi.utilities.HDFSParquetImporter;
import org.apache.hudi.utilities.HDFSParquetImporter.Config;
import org.apache.hudi.utilities.HoodieCleaner;
+import org.apache.hudi.utilities.HoodieClusteringJob;
import org.apache.hudi.utilities.HoodieCompactionAdminTool;
import org.apache.hudi.utilities.HoodieCompactionAdminTool.Operation;
import org.apache.hudi.utilities.HoodieCompactor;
-import org.apache.hudi.utilities.UtilHelpers;
import org.apache.hudi.utilities.deltastreamer.BootstrapExecutor;
import org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer;
+import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
-import org.apache.log4j.Logger;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
import org.apache.spark.sql.SQLContext;
+import org.apache.spark.sql.functions;
+import org.apache.spark.sql.types.StructType;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
+import java.util.HashMap;
import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+
+import static org.apache.hudi.utilities.UtilHelpers.EXECUTE;
+import static org.apache.hudi.utilities.UtilHelpers.SCHEDULE;
+import static org.apache.hudi.utilities.UtilHelpers.SCHEDULE_AND_EXECUTE;
+import static org.apache.hudi.utilities.UtilHelpers.buildProperties;
+import static org.apache.hudi.utilities.UtilHelpers.readConfig;
/**
* This class deals with initializing spark context based on command entered to hudi-cli.
*/
public class SparkMain {
- private static final Logger LOG = Logger.getLogger(SparkMain.class);
+ private static final Logger LOG = LogManager.getLogger(SparkMain.class);
/**
* Commands.
*/
enum SparkCommand {
- BOOTSTRAP, ROLLBACK, DEDUPLICATE, ROLLBACK_TO_SAVEPOINT, SAVEPOINT, IMPORT, UPSERT, COMPACT_SCHEDULE, COMPACT_RUN,
- COMPACT_UNSCHEDULE_PLAN, COMPACT_UNSCHEDULE_FILE, COMPACT_VALIDATE, COMPACT_REPAIR, CLEAN, DELETE_SAVEPOINT, UPGRADE, DOWNGRADE
+ BOOTSTRAP, ROLLBACK, DEDUPLICATE, ROLLBACK_TO_SAVEPOINT, SAVEPOINT, IMPORT, UPSERT, COMPACT_SCHEDULE, COMPACT_RUN, COMPACT_SCHEDULE_AND_EXECUTE,
+ COMPACT_UNSCHEDULE_PLAN, COMPACT_UNSCHEDULE_FILE, COMPACT_VALIDATE, COMPACT_REPAIR, CLUSTERING_SCHEDULE,
+ CLUSTERING_RUN, CLUSTERING_SCHEDULE_AND_EXECUTE, CLEAN, DELETE_MARKER, DELETE_SAVEPOINT, UPGRADE, DOWNGRADE,
+ REPAIR_DEPRECATED_PARTITION, RENAME_PARTITION
}
public static void main(String[] args) throws Exception {
- String command = args[0];
- LOG.info("Invoking SparkMain:" + command);
+ ValidationUtils.checkArgument(args.length >= 4);
+ final String commandString = args[0];
+ LOG.info("Invoking SparkMain: " + commandString);
+ final SparkCommand cmd = SparkCommand.valueOf(commandString);
- SparkCommand cmd = SparkCommand.valueOf(command);
+ JavaSparkContext jsc = SparkUtil.initJavaSparkContext("hoodie-cli-" + commandString,
+ Option.of(args[1]), Option.of(args[2]));
- JavaSparkContext jsc = sparkMasterContained(cmd)
- ? SparkUtil.initJavaSparkConf("hoodie-cli-" + command, Option.of(args[1]), Option.of(args[2]))
- : SparkUtil.initJavaSparkConf("hoodie-cli-" + command);
int returnCode = 0;
- switch (cmd) {
- case ROLLBACK:
- assert (args.length == 5);
- returnCode = rollback(jsc, args[3], args[4]);
- break;
- case DEDUPLICATE:
- assert (args.length == 8);
- returnCode = deduplicatePartitionPath(jsc, args[3], args[4], args[5], Boolean.parseBoolean(args[6]), args[7]);
- break;
- case ROLLBACK_TO_SAVEPOINT:
- assert (args.length == 5);
- returnCode = rollbackToSavepoint(jsc, args[3], args[4]);
- break;
- case IMPORT:
- case UPSERT:
- assert (args.length >= 13);
- String propsFilePath = null;
- if (!StringUtils.isNullOrEmpty(args[12])) {
- propsFilePath = args[12];
- }
- List configs = new ArrayList<>();
- if (args.length > 13) {
- configs.addAll(Arrays.asList(args).subList(13, args.length));
- }
- returnCode = dataLoad(jsc, command, args[3], args[4], args[5], args[6], args[7], args[8],
- Integer.parseInt(args[9]), args[10], Integer.parseInt(args[11]), propsFilePath, configs);
- break;
- case COMPACT_RUN:
- assert (args.length >= 9);
- propsFilePath = null;
- if (!StringUtils.isNullOrEmpty(args[8])) {
- propsFilePath = args[8];
- }
- configs = new ArrayList<>();
- if (args.length > 9) {
- configs.addAll(Arrays.asList(args).subList(9, args.length));
- }
- returnCode = compact(jsc, args[1], args[2], args[3], Integer.parseInt(args[4]), args[5], args[6],
- Integer.parseInt(args[7]), false, propsFilePath, configs);
- break;
- case COMPACT_SCHEDULE:
- assert (args.length >= 6);
- propsFilePath = null;
- if (!StringUtils.isNullOrEmpty(args[5])) {
- propsFilePath = args[5];
- }
- configs = new ArrayList<>();
- if (args.length > 6) {
- configs.addAll(Arrays.asList(args).subList(6, args.length));
- }
- returnCode = compact(jsc, args[1], args[2], args[3], 1, "", args[4], 0, true, propsFilePath, configs);
- break;
- case COMPACT_VALIDATE:
- assert (args.length == 7);
- doCompactValidate(jsc, args[3], args[4], args[5], Integer.parseInt(args[6]));
- returnCode = 0;
- break;
- case COMPACT_REPAIR:
- assert (args.length == 8);
- doCompactRepair(jsc, args[3], args[4], args[5], Integer.parseInt(args[6]),
- Boolean.parseBoolean(args[7]));
- returnCode = 0;
- break;
- case COMPACT_UNSCHEDULE_FILE:
- assert (args.length == 9);
- doCompactUnscheduleFile(jsc, args[3], args[4], args[5], Integer.parseInt(args[6]),
- Boolean.parseBoolean(args[7]), Boolean.parseBoolean(args[8]));
- returnCode = 0;
- break;
- case COMPACT_UNSCHEDULE_PLAN:
- assert (args.length == 9);
- doCompactUnschedule(jsc, args[3], args[4], args[5], Integer.parseInt(args[6]),
- Boolean.parseBoolean(args[7]), Boolean.parseBoolean(args[8]));
- returnCode = 0;
- break;
- case CLEAN:
- assert (args.length >= 5);
- propsFilePath = null;
- if (!StringUtils.isNullOrEmpty(args[4])) {
- propsFilePath = args[4];
- }
- configs = new ArrayList<>();
- if (args.length > 5) {
- configs.addAll(Arrays.asList(args).subList(5, args.length));
- }
- clean(jsc, args[3], propsFilePath, configs);
- break;
- case SAVEPOINT:
- assert (args.length == 7);
- returnCode = createSavepoint(jsc, args[3], args[4], args[5], args[6]);
- break;
- case DELETE_SAVEPOINT:
- assert (args.length == 5);
- returnCode = deleteSavepoint(jsc, args[3], args[4]);
- break;
- case BOOTSTRAP:
- assert (args.length >= 18);
- propsFilePath = null;
- if (!StringUtils.isNullOrEmpty(args[17])) {
- propsFilePath = args[17];
- }
- configs = new ArrayList<>();
- if (args.length > 18) {
- configs.addAll(Arrays.asList(args).subList(18, args.length));
- }
- returnCode = doBootstrap(jsc, args[3], args[4], args[5], args[6], args[7], args[8], args[9], args[10],
- args[11], args[12], args[13], args[14], args[15], args[16], propsFilePath, configs);
- break;
- case UPGRADE:
- case DOWNGRADE:
- assert (args.length == 5);
- returnCode = upgradeOrDowngradeTable(jsc, args[3], args[4]);
- break;
- default:
- break;
+ try {
+ switch (cmd) {
+ case ROLLBACK:
+ assert (args.length == 6);
+ returnCode = rollback(jsc, args[3], args[4], Boolean.parseBoolean(args[5]));
+ break;
+ case DEDUPLICATE:
+ assert (args.length == 8);
+ returnCode = deduplicatePartitionPath(jsc, args[3], args[4], args[5], Boolean.parseBoolean(args[6]), args[7]);
+ break;
+ case ROLLBACK_TO_SAVEPOINT:
+ assert (args.length == 6);
+ returnCode = rollbackToSavepoint(jsc, args[3], args[4], Boolean.parseBoolean(args[5]));
+ break;
+ case IMPORT:
+ case UPSERT:
+ assert (args.length >= 13);
+ String propsFilePath = null;
+ if (!StringUtils.isNullOrEmpty(args[12])) {
+ propsFilePath = args[12];
+ }
+ List configs = new ArrayList<>();
+ if (args.length > 13) {
+ configs.addAll(Arrays.asList(args).subList(13, args.length));
+ }
+ returnCode = dataLoad(jsc, commandString, args[3], args[4], args[5], args[6], args[7], args[8],
+ Integer.parseInt(args[9]), args[10], Integer.parseInt(args[11]), propsFilePath, configs);
+ break;
+ case COMPACT_RUN:
+ assert (args.length >= 10);
+ propsFilePath = null;
+ if (!StringUtils.isNullOrEmpty(args[9])) {
+ propsFilePath = args[9];
+ }
+ configs = new ArrayList<>();
+ if (args.length > 10) {
+ configs.addAll(Arrays.asList(args).subList(10, args.length));
+ }
+ returnCode = compact(jsc, args[3], args[4], args[5], Integer.parseInt(args[6]), args[7],
+ Integer.parseInt(args[8]), HoodieCompactor.EXECUTE, propsFilePath, configs);
+ break;
+ case COMPACT_SCHEDULE_AND_EXECUTE:
+ assert (args.length >= 9);
+ propsFilePath = null;
+ if (!StringUtils.isNullOrEmpty(args[8])) {
+ propsFilePath = args[8];
+ }
+ configs = new ArrayList<>();
+ if (args.length > 9) {
+ configs.addAll(Arrays.asList(args).subList(9, args.length));
+ }
+
+ returnCode = compact(jsc, args[3], args[4], null, Integer.parseInt(args[5]), args[6],
+ Integer.parseInt(args[7]), HoodieCompactor.SCHEDULE_AND_EXECUTE, propsFilePath, configs);
+ break;
+ case COMPACT_SCHEDULE:
+ assert (args.length >= 7);
+ propsFilePath = null;
+ if (!StringUtils.isNullOrEmpty(args[6])) {
+ propsFilePath = args[6];
+ }
+ configs = new ArrayList<>();
+ if (args.length > 7) {
+ configs.addAll(Arrays.asList(args).subList(7, args.length));
+ }
+ returnCode = compact(jsc, args[3], args[4], args[5], 1, "", 0, HoodieCompactor.SCHEDULE, propsFilePath, configs);
+ break;
+ case COMPACT_VALIDATE:
+ assert (args.length == 7);
+ doCompactValidate(jsc, args[3], args[4], args[5], Integer.parseInt(args[6]));
+ returnCode = 0;
+ break;
+ case COMPACT_REPAIR:
+ assert (args.length == 8);
+ doCompactRepair(jsc, args[3], args[4], args[5], Integer.parseInt(args[6]),
+ Boolean.parseBoolean(args[7]));
+ returnCode = 0;
+ break;
+ case COMPACT_UNSCHEDULE_FILE:
+ assert (args.length == 10);
+ doCompactUnscheduleFile(jsc, args[3], args[4], args[5], args[6], Integer.parseInt(args[7]),
+ Boolean.parseBoolean(args[8]), Boolean.parseBoolean(args[9]));
+ returnCode = 0;
+ break;
+ case COMPACT_UNSCHEDULE_PLAN:
+ assert (args.length == 9);
+ doCompactUnschedule(jsc, args[3], args[4], args[5], Integer.parseInt(args[6]),
+ Boolean.parseBoolean(args[7]), Boolean.parseBoolean(args[8]));
+ returnCode = 0;
+ break;
+ case CLUSTERING_RUN:
+ assert (args.length >= 9);
+ propsFilePath = null;
+ if (!StringUtils.isNullOrEmpty(args[8])) {
+ propsFilePath = args[8];
+ }
+ configs = new ArrayList<>();
+ if (args.length > 9) {
+ configs.addAll(Arrays.asList(args).subList(9, args.length));
+ }
+ returnCode = cluster(jsc, args[3], args[4], args[5], Integer.parseInt(args[6]), args[2],
+ Integer.parseInt(args[7]), EXECUTE, propsFilePath, configs);
+ break;
+ case CLUSTERING_SCHEDULE_AND_EXECUTE:
+ assert (args.length >= 8);
+ propsFilePath = null;
+ if (!StringUtils.isNullOrEmpty(args[7])) {
+ propsFilePath = args[7];
+ }
+ configs = new ArrayList<>();
+ if (args.length > 8) {
+ configs.addAll(Arrays.asList(args).subList(8, args.length));
+ }
+ returnCode = cluster(jsc, args[3], args[4], null, Integer.parseInt(args[5]), args[2],
+ Integer.parseInt(args[6]), SCHEDULE_AND_EXECUTE, propsFilePath, configs);
+ break;
+ case CLUSTERING_SCHEDULE:
+ assert (args.length >= 7);
+ propsFilePath = null;
+ if (!StringUtils.isNullOrEmpty(args[6])) {
+ propsFilePath = args[6];
+ }
+ configs = new ArrayList<>();
+ if (args.length > 7) {
+ configs.addAll(Arrays.asList(args).subList(7, args.length));
+ }
+ returnCode = cluster(jsc, args[3], args[4], args[5], 1, args[2],
+ 0, SCHEDULE, propsFilePath, configs);
+ break;
+ case CLEAN:
+ assert (args.length >= 5);
+ propsFilePath = null;
+ if (!StringUtils.isNullOrEmpty(args[4])) {
+ propsFilePath = args[4];
+ }
+ configs = new ArrayList<>();
+ if (args.length > 5) {
+ configs.addAll(Arrays.asList(args).subList(5, args.length));
+ }
+ clean(jsc, args[3], propsFilePath, configs);
+ break;
+ case SAVEPOINT:
+ assert (args.length == 7);
+ returnCode = createSavepoint(jsc, args[3], args[4], args[5], args[6]);
+ break;
+ case DELETE_MARKER:
+ assert (args.length == 5);
+ returnCode = deleteMarker(jsc, args[3], args[4]);
+ break;
+ case DELETE_SAVEPOINT:
+ assert (args.length == 5);
+ returnCode = deleteSavepoint(jsc, args[3], args[4]);
+ break;
+ case BOOTSTRAP:
+ assert (args.length >= 18);
+ propsFilePath = null;
+ if (!StringUtils.isNullOrEmpty(args[17])) {
+ propsFilePath = args[17];
+ }
+ configs = new ArrayList<>();
+ if (args.length > 18) {
+ configs.addAll(Arrays.asList(args).subList(18, args.length));
+ }
+ returnCode = doBootstrap(jsc, args[3], args[4], args[5], args[6], args[7], args[8], args[9], args[10],
+ args[11], args[12], args[13], args[14], args[15], args[16], propsFilePath, configs);
+ break;
+ case UPGRADE:
+ case DOWNGRADE:
+ assert (args.length == 5);
+ returnCode = upgradeOrDowngradeTable(jsc, args[3], args[4]);
+ break;
+ case REPAIR_DEPRECATED_PARTITION:
+ assert (args.length == 4);
+ returnCode = repairDeprecatedPartition(jsc, args[3]);
+ break;
+ case RENAME_PARTITION:
+ assert (args.length == 6);
+ returnCode = renamePartition(jsc, args[3], args[4], args[5]);
+ break;
+ default:
+ break;
+ }
+ } catch (Throwable throwable) {
+ LOG.error("Fail to execute commandString", throwable);
+ returnCode = -1;
+ } finally {
+ jsc.stop();
}
System.exit(returnCode);
}
- private static boolean sparkMasterContained(SparkCommand command) {
- List masterContained = Arrays.asList(SparkCommand.COMPACT_VALIDATE, SparkCommand.COMPACT_REPAIR,
- SparkCommand.COMPACT_UNSCHEDULE_PLAN, SparkCommand.COMPACT_UNSCHEDULE_FILE, SparkCommand.CLEAN,
- SparkCommand.IMPORT, SparkCommand.UPSERT, SparkCommand.DEDUPLICATE, SparkCommand.SAVEPOINT,
- SparkCommand.DELETE_SAVEPOINT, SparkCommand.ROLLBACK_TO_SAVEPOINT, SparkCommand.ROLLBACK, SparkCommand.BOOTSTRAP);
- return masterContained.contains(command);
- }
-
protected static void clean(JavaSparkContext jsc, String basePath, String propsFilePath,
- List configs) {
+ List configs) {
HoodieCleaner.Config cfg = new HoodieCleaner.Config();
cfg.basePath = basePath;
cfg.propsFilePath = propsFilePath;
@@ -219,9 +311,24 @@ protected static void clean(JavaSparkContext jsc, String basePath, String propsF
new HoodieCleaner(cfg, jsc).run();
}
+ protected static int deleteMarker(JavaSparkContext jsc, String instantTime, String basePath) {
+ try {
+ SparkRDDWriteClient client = createHoodieClient(jsc, basePath, false);
+ HoodieWriteConfig config = client.getConfig();
+ HoodieEngineContext context = client.getEngineContext();
+ HoodieSparkTable table = HoodieSparkTable.create(config, context);
+ WriteMarkersFactory.get(config.getMarkersType(), table, instantTime)
+ .quietDeleteMarkerDir(context, config.getMarkersDeleteParallelism());
+ return 0;
+ } catch (Exception e) {
+ LOG.warn(String.format("Failed: Could not clean marker instantTime: \"%s\".", instantTime), e);
+ return -1;
+ }
+ }
+
private static int dataLoad(JavaSparkContext jsc, String command, String srcPath, String targetPath, String tableName,
- String tableType, String rowKey, String partitionKey, int parallelism, String schemaFile,
- int retry, String propsFilePath, List configs) {
+ String tableType, String rowKey, String partitionKey, int parallelism, String schemaFile,
+ int retry, String propsFilePath, List configs) {
Config cfg = new Config();
cfg.command = command;
cfg.srcPath = srcPath;
@@ -238,7 +345,7 @@ private static int dataLoad(JavaSparkContext jsc, String command, String srcPath
}
private static void doCompactValidate(JavaSparkContext jsc, String basePath, String compactionInstant,
- String outputPath, int parallelism) throws Exception {
+ String outputPath, int parallelism) throws Exception {
HoodieCompactionAdminTool.Config cfg = new HoodieCompactionAdminTool.Config();
cfg.basePath = basePath;
cfg.operation = Operation.VALIDATE;
@@ -249,7 +356,7 @@ private static void doCompactValidate(JavaSparkContext jsc, String basePath, Str
}
private static void doCompactRepair(JavaSparkContext jsc, String basePath, String compactionInstant,
- String outputPath, int parallelism, boolean dryRun) throws Exception {
+ String outputPath, int parallelism, boolean dryRun) throws Exception {
HoodieCompactionAdminTool.Config cfg = new HoodieCompactionAdminTool.Config();
cfg.basePath = basePath;
cfg.operation = Operation.REPAIR;
@@ -261,7 +368,7 @@ private static void doCompactRepair(JavaSparkContext jsc, String basePath, Strin
}
private static void doCompactUnschedule(JavaSparkContext jsc, String basePath, String compactionInstant,
- String outputPath, int parallelism, boolean skipValidation, boolean dryRun) throws Exception {
+ String outputPath, int parallelism, boolean skipValidation, boolean dryRun) throws Exception {
HoodieCompactionAdminTool.Config cfg = new HoodieCompactionAdminTool.Config();
cfg.basePath = basePath;
cfg.operation = Operation.UNSCHEDULE_PLAN;
@@ -273,13 +380,14 @@ private static void doCompactUnschedule(JavaSparkContext jsc, String basePath, S
new HoodieCompactionAdminTool(cfg).run(jsc);
}
- private static void doCompactUnscheduleFile(JavaSparkContext jsc, String basePath, String fileId, String outputPath,
- int parallelism, boolean skipValidation, boolean dryRun)
+ private static void doCompactUnscheduleFile(JavaSparkContext jsc, String basePath, String fileId, String partitionPath,
+ String outputPath, int parallelism, boolean skipValidation, boolean dryRun)
throws Exception {
HoodieCompactionAdminTool.Config cfg = new HoodieCompactionAdminTool.Config();
cfg.basePath = basePath;
cfg.operation = Operation.UNSCHEDULE_FILE;
cfg.outputPath = outputPath;
+ cfg.partitionPath = partitionPath;
cfg.fileId = fileId;
cfg.parallelism = parallelism;
cfg.dryRun = dryRun;
@@ -288,8 +396,8 @@ private static void doCompactUnscheduleFile(JavaSparkContext jsc, String basePat
}
private static int compact(JavaSparkContext jsc, String basePath, String tableName, String compactionInstant,
- int parallelism, String schemaFile, String sparkMemory, int retry, boolean schedule, String propsFilePath,
- List configs) {
+ int parallelism, String schemaFile, int retry, String mode, String propsFilePath,
+ List configs) {
HoodieCompactor.Config cfg = new HoodieCompactor.Config();
cfg.basePath = basePath;
cfg.tableName = tableName;
@@ -298,36 +406,137 @@ private static int compact(JavaSparkContext jsc, String basePath, String tableNa
cfg.strategyClassName = UnBoundedCompactionStrategy.class.getCanonicalName();
cfg.parallelism = parallelism;
cfg.schemaFile = schemaFile;
- cfg.runSchedule = schedule;
+ cfg.runningMode = mode;
cfg.propsFilePath = propsFilePath;
cfg.configs = configs;
- jsc.getConf().set("spark.executor.memory", sparkMemory);
return new HoodieCompactor(jsc, cfg).compact(retry);
}
+ private static int cluster(JavaSparkContext jsc, String basePath, String tableName, String clusteringInstant,
+ int parallelism, String sparkMemory, int retry, String runningMode, String propsFilePath, List configs) {
+ HoodieClusteringJob.Config cfg = new HoodieClusteringJob.Config();
+ cfg.basePath = basePath;
+ cfg.tableName = tableName;
+ cfg.clusteringInstantTime = clusteringInstant;
+ cfg.parallelism = parallelism;
+ cfg.runningMode = runningMode;
+ cfg.propsFilePath = propsFilePath;
+ cfg.configs = configs;
+ jsc.getConf().set("spark.executor.memory", sparkMemory);
+ return new HoodieClusteringJob(jsc, cfg).cluster(retry);
+ }
+
private static int deduplicatePartitionPath(JavaSparkContext jsc, String duplicatedPartitionPath,
- String repairedOutputPath, String basePath, boolean dryRun, String dedupeType) {
+ String repairedOutputPath, String basePath, boolean dryRun, String dedupeType) {
DedupeSparkJob job = new DedupeSparkJob(basePath, duplicatedPartitionPath, repairedOutputPath, new SQLContext(jsc),
FSUtils.getFs(basePath, jsc.hadoopConfiguration()), DeDupeType.withName(dedupeType));
job.fixDuplicates(dryRun);
return 0;
}
+ public static int repairDeprecatedPartition(JavaSparkContext jsc, String basePath) {
+ SQLContext sqlContext = new SQLContext(jsc);
+ Dataset recordsToRewrite = getRecordsToRewrite(basePath, PartitionPathEncodeUtils.DEPRECATED_DEFAULT_PARTITION_PATH, sqlContext);
+
+ if (!recordsToRewrite.isEmpty()) {
+ recordsToRewrite.cache();
+ HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setConf(jsc.hadoopConfiguration()).setBasePath(basePath).build();
+ Map propsMap = getPropsForRewrite(metaClient);
+ rewriteRecordsToNewPartition(basePath, PartitionPathEncodeUtils.DEFAULT_PARTITION_PATH, recordsToRewrite, metaClient, propsMap);
+ // after re-writing, we can safely delete older data.
+ deleteOlderPartition(basePath, PartitionPathEncodeUtils.DEPRECATED_DEFAULT_PARTITION_PATH, recordsToRewrite, propsMap);
+ }
+ return 0;
+ }
+
+ public static int renamePartition(JavaSparkContext jsc, String basePath, String oldPartition, String newPartition) {
+ SQLContext sqlContext = new SQLContext(jsc);
+ Dataset recordsToRewrite = getRecordsToRewrite(basePath, oldPartition, sqlContext);
+
+ if (!recordsToRewrite.isEmpty()) {
+ recordsToRewrite.cache();
+ HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setConf(jsc.hadoopConfiguration()).setBasePath(basePath).build();
+ Map propsMap = getPropsForRewrite(metaClient);
+ rewriteRecordsToNewPartition(basePath, newPartition, recordsToRewrite, metaClient, propsMap);
+ // after re-writing, we can safely delete older partition.
+ deleteOlderPartition(basePath, oldPartition, recordsToRewrite, propsMap);
+ // also, we can physically delete the old partition.
+ FileSystem fs = FSUtils.getFs(new Path(basePath), metaClient.getHadoopConf());
+ try {
+ fs.delete(new Path(basePath, oldPartition), true);
+ } catch (IOException e) {
+ LOG.warn("Failed to delete older partition " + basePath);
+ }
+ }
+ return 0;
+ }
+
+ private static void deleteOlderPartition(String basePath, String oldPartition, Dataset recordsToRewrite, Map propsMap) {
+ propsMap.put("hoodie.datasource.write.partitions.to.delete", oldPartition);
+ recordsToRewrite.write()
+ .options(propsMap)
+ .option("hoodie.datasource.write.operation", WriteOperationType.DELETE_PARTITION.value())
+ .format("hudi")
+ .mode("Append")
+ .save(basePath);
+ }
+
+ private static void rewriteRecordsToNewPartition(String basePath, String newPartition, Dataset recordsToRewrite, HoodieTableMetaClient metaClient, Map propsMap) {
+ String partitionFieldProp = metaClient.getTableConfig().getPartitionFieldProp();
+ StructType structType = recordsToRewrite.schema();
+ int partitionIndex = structType.fieldIndex(partitionFieldProp);
+
+ recordsToRewrite.withColumn(metaClient.getTableConfig().getPartitionFieldProp(), functions.lit(null).cast(structType.apply(partitionIndex).dataType()))
+ .write()
+ .options(propsMap)
+ .option("hoodie.datasource.write.operation", WriteOperationType.BULK_INSERT.value())
+ .format("hudi")
+ .mode("Append")
+ .save(basePath);
+ }
+
+ private static Dataset getRecordsToRewrite(String basePath, String oldPartition, SQLContext sqlContext) {
+ return sqlContext.read()
+ .format("hudi")
+ .load(basePath + "/" + oldPartition)
+ .drop(HoodieRecord.RECORD_KEY_METADATA_FIELD)
+ .drop(HoodieRecord.PARTITION_PATH_METADATA_FIELD)
+ .drop(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD)
+ .drop(HoodieRecord.FILENAME_METADATA_FIELD)
+ .drop(HoodieRecord.COMMIT_TIME_METADATA_FIELD);
+ }
+
+ private static Map getPropsForRewrite(HoodieTableMetaClient metaClient) {
+ Map propsMap = new HashMap<>();
+ metaClient.getTableConfig().getProps().forEach((k, v) -> propsMap.put(k.toString(), v.toString()));
+ propsMap.put(HoodieWriteConfig.SKIP_DEFAULT_PARTITION_VALIDATION.key(), "true");
+ propsMap.put(DataSourceWriteOptions.RECORDKEY_FIELD().key(), metaClient.getTableConfig().getRecordKeyFieldProp());
+ propsMap.put(DataSourceWriteOptions.PARTITIONPATH_FIELD().key(), metaClient.getTableConfig().getPartitionFieldProp());
+ propsMap.put(DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME().key(), metaClient.getTableConfig().getKeyGeneratorClassName());
+ return propsMap;
+ }
+
private static int doBootstrap(JavaSparkContext jsc, String tableName, String tableType, String basePath,
- String sourcePath, String recordKeyCols, String partitionFields, String parallelism, String schemaProviderClass,
- String bootstrapIndexClass, String selectorClass, String keyGeneratorClass, String fullBootstrapInputProvider,
- String payloadClassName, String enableHiveSync, String propsFilePath, List configs) throws IOException {
+ String sourcePath, String recordKeyCols, String partitionFields, String parallelism, String schemaProviderClass,
+ String bootstrapIndexClass, String selectorClass, String keyGenerator, String fullBootstrapInputProvider,
+ String payloadClassName, String enableHiveSync, String propsFilePath, List configs) throws IOException {
- TypedProperties properties = propsFilePath == null ? UtilHelpers.buildProperties(configs)
- : UtilHelpers.readConfig(FSUtils.getFs(propsFilePath, jsc.hadoopConfiguration()), new Path(propsFilePath), configs).getConfig();
+ TypedProperties properties = propsFilePath == null ? buildProperties(configs)
+ : readConfig(jsc.hadoopConfiguration(), new Path(propsFilePath), configs).getProps(true);
- properties.setProperty(HoodieBootstrapConfig.BOOTSTRAP_BASE_PATH_PROP, sourcePath);
- properties.setProperty(HoodieBootstrapConfig.BOOTSTRAP_KEYGEN_CLASS, keyGeneratorClass);
- properties.setProperty(HoodieBootstrapConfig.FULL_BOOTSTRAP_INPUT_PROVIDER, fullBootstrapInputProvider);
- properties.setProperty(HoodieBootstrapConfig.BOOTSTRAP_PARALLELISM, parallelism);
- properties.setProperty(HoodieBootstrapConfig.BOOTSTRAP_MODE_SELECTOR, selectorClass);
- properties.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY(), recordKeyCols);
- properties.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY(), partitionFields);
+ properties.setProperty(HoodieBootstrapConfig.BASE_PATH.key(), sourcePath);
+
+ if (!StringUtils.isNullOrEmpty(keyGenerator) && KeyGeneratorType.getNames().contains(keyGenerator.toUpperCase(Locale.ROOT))) {
+ properties.setProperty(HoodieBootstrapConfig.KEYGEN_TYPE.key(), keyGenerator.toUpperCase(Locale.ROOT));
+ } else {
+ properties.setProperty(HoodieBootstrapConfig.KEYGEN_CLASS_NAME.key(), keyGenerator);
+ }
+
+ properties.setProperty(HoodieBootstrapConfig.FULL_BOOTSTRAP_INPUT_PROVIDER_CLASS_NAME.key(), fullBootstrapInputProvider);
+ properties.setProperty(HoodieBootstrapConfig.PARALLELISM_VALUE.key(), parallelism);
+ properties.setProperty(HoodieBootstrapConfig.MODE_SELECTOR_CLASS_NAME.key(), selectorClass);
+ properties.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD().key(), recordKeyCols);
+ properties.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD().key(), partitionFields);
HoodieDeltaStreamer.Config cfg = new HoodieDeltaStreamer.Config();
cfg.targetTableName = tableName;
@@ -343,8 +552,8 @@ private static int doBootstrap(JavaSparkContext jsc, String tableName, String ta
return 0;
}
- private static int rollback(JavaSparkContext jsc, String instantTime, String basePath) throws Exception {
- SparkRDDWriteClient client = createHoodieClient(jsc, basePath);
+ private static int rollback(JavaSparkContext jsc, String instantTime, String basePath, Boolean rollbackUsingMarkers) throws Exception {
+ SparkRDDWriteClient client = createHoodieClient(jsc, basePath, rollbackUsingMarkers, false);
if (client.rollback(instantTime)) {
LOG.info(String.format("The commit \"%s\" rolled back.", instantTime));
return 0;
@@ -355,8 +564,8 @@ private static int rollback(JavaSparkContext jsc, String instantTime, String bas
}
private static int createSavepoint(JavaSparkContext jsc, String commitTime, String user,
- String comments, String basePath) throws Exception {
- SparkRDDWriteClient client = createHoodieClient(jsc, basePath);
+ String comments, String basePath) throws Exception {
+ SparkRDDWriteClient client = createHoodieClient(jsc, basePath, false);
try {
client.savepoint(commitTime, user, comments);
LOG.info(String.format("The commit \"%s\" has been savepointed.", commitTime));
@@ -367,26 +576,26 @@ private static int createSavepoint(JavaSparkContext jsc, String commitTime, Stri
}
}
- private static int rollbackToSavepoint(JavaSparkContext jsc, String savepointTime, String basePath) throws Exception {
- SparkRDDWriteClient client = createHoodieClient(jsc, basePath);
+ private static int rollbackToSavepoint(JavaSparkContext jsc, String savepointTime, String basePath, boolean lazyCleanPolicy) throws Exception {
+ SparkRDDWriteClient client = createHoodieClient(jsc, basePath, lazyCleanPolicy);
try {
client.restoreToSavepoint(savepointTime);
LOG.info(String.format("The commit \"%s\" rolled back.", savepointTime));
return 0;
} catch (Exception e) {
- LOG.warn(String.format("The commit \"%s\" failed to roll back.", savepointTime));
+ LOG.warn(String.format("The commit \"%s\" failed to roll back.", savepointTime), e);
return -1;
}
}
private static int deleteSavepoint(JavaSparkContext jsc, String savepointTime, String basePath) throws Exception {
- SparkRDDWriteClient client = createHoodieClient(jsc, basePath);
+ SparkRDDWriteClient client = createHoodieClient(jsc, basePath, false);
try {
client.deleteSavepoint(savepointTime);
LOG.info(String.format("Savepoint \"%s\" deleted.", savepointTime));
return 0;
} catch (Exception e) {
- LOG.warn(String.format("Failed: Could not delete savepoint \"%s\".", savepointTime));
+ LOG.warn(String.format("Failed: Could not delete savepoint \"%s\".", savepointTime), e);
return -1;
}
}
@@ -394,18 +603,25 @@ private static int deleteSavepoint(JavaSparkContext jsc, String savepointTime, S
/**
* Upgrade or downgrade table.
*
- * @param jsc instance of {@link JavaSparkContext} to use.
- * @param basePath base path of the dataset.
+ * @param jsc instance of {@link JavaSparkContext} to use.
+ * @param basePath base path of the dataset.
* @param toVersion version to which upgrade/downgrade to be done.
* @return 0 if success, else -1.
* @throws Exception
*/
protected static int upgradeOrDowngradeTable(JavaSparkContext jsc, String basePath, String toVersion) {
- HoodieWriteConfig config = getWriteConfig(basePath);
- HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), false,
- config.getConsistencyGuardConfig(), Option.of(new TimelineLayoutVersion(config.getTimelineLayoutVersion())));
+ HoodieWriteConfig config = getWriteConfig(basePath, Boolean.parseBoolean(HoodieWriteConfig.ROLLBACK_USING_MARKERS_ENABLE.defaultValue()),
+ false);
+ HoodieTableMetaClient metaClient =
+ HoodieTableMetaClient.builder().setConf(jsc.hadoopConfiguration()).setBasePath(config.getBasePath())
+ .setLoadActiveTimelineOnLoad(false).setConsistencyGuardConfig(config.getConsistencyGuardConfig())
+ .setLayoutVersion(Option.of(new TimelineLayoutVersion(config.getTimelineLayoutVersion())))
+ .setFileSystemRetryConfig(config.getFileSystemRetryConfig()).build();
+ HoodieWriteConfig updatedConfig = HoodieWriteConfig.newBuilder().withProps(config.getProps())
+ .forTable(metaClient.getTableConfig().getTableName()).build();
try {
- new SparkUpgradeDowngrade(metaClient, config, new HoodieSparkEngineContext(jsc)).run(metaClient, HoodieTableVersion.valueOf(toVersion), config, new HoodieSparkEngineContext(jsc), null);
+ new UpgradeDowngrade(metaClient, updatedConfig, new HoodieSparkEngineContext(jsc), SparkUpgradeDowngradeHelper.getInstance())
+ .run(HoodieTableVersion.valueOf(toVersion), null);
LOG.info(String.format("Table at \"%s\" upgraded / downgraded to version \"%s\".", basePath, toVersion));
return 0;
} catch (Exception e) {
@@ -414,13 +630,20 @@ protected static int upgradeOrDowngradeTable(JavaSparkContext jsc, String basePa
}
}
- private static SparkRDDWriteClient createHoodieClient(JavaSparkContext jsc, String basePath) throws Exception {
- HoodieWriteConfig config = getWriteConfig(basePath);
+ private static SparkRDDWriteClient createHoodieClient(JavaSparkContext jsc, String basePath, Boolean rollbackUsingMarkers, boolean lazyCleanPolicy) throws Exception {
+ HoodieWriteConfig config = getWriteConfig(basePath, rollbackUsingMarkers, lazyCleanPolicy);
return new SparkRDDWriteClient(new HoodieSparkEngineContext(jsc), config);
}
- private static HoodieWriteConfig getWriteConfig(String basePath) {
+ private static SparkRDDWriteClient createHoodieClient(JavaSparkContext jsc, String basePath, boolean lazyCleanPolicy) throws Exception {
+ return createHoodieClient(jsc, basePath, Boolean.parseBoolean(HoodieWriteConfig.ROLLBACK_USING_MARKERS_ENABLE.defaultValue()), lazyCleanPolicy);
+ }
+
+ private static HoodieWriteConfig getWriteConfig(String basePath, Boolean rollbackUsingMarkers, boolean lazyCleanPolicy) {
return HoodieWriteConfig.newBuilder().withPath(basePath)
+ .withRollbackUsingMarkers(rollbackUsingMarkers)
+ .withCleanConfig(HoodieCleanConfig.newBuilder().withFailedWritesCleaningPolicy(lazyCleanPolicy ? HoodieFailedWritesCleaningPolicy.LAZY :
+ HoodieFailedWritesCleaningPolicy.EAGER).build())
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()).build();
}
}
diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/StatsCommand.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/StatsCommand.java
index 66c5563102848..c9034d03d5fcd 100644
--- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/StatsCommand.java
+++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/StatsCommand.java
@@ -18,6 +18,12 @@
package org.apache.hudi.cli.commands;
+import com.codahale.metrics.Histogram;
+import com.codahale.metrics.Snapshot;
+import com.codahale.metrics.UniformReservoir;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
import org.apache.hudi.cli.HoodieCLI;
import org.apache.hudi.cli.HoodiePrintHelper;
import org.apache.hudi.cli.HoodieTableHeaderFields;
@@ -28,17 +34,9 @@
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.common.util.NumericUtils;
-
-import com.codahale.metrics.Histogram;
-import com.codahale.metrics.Snapshot;
-import com.codahale.metrics.UniformReservoir;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.springframework.shell.core.CommandMarker;
-import org.springframework.shell.core.annotation.CliCommand;
-import org.springframework.shell.core.annotation.CliOption;
-import org.springframework.stereotype.Component;
+import org.springframework.shell.standard.ShellComponent;
+import org.springframework.shell.standard.ShellMethod;
+import org.springframework.shell.standard.ShellOption;
import java.io.IOException;
import java.text.DecimalFormat;
@@ -52,19 +50,19 @@
/**
* CLI command to displays stats options.
*/
-@Component
-public class StatsCommand implements CommandMarker {
+@ShellComponent
+public class StatsCommand {
public static final int MAX_FILES = 1000000;
- @CliCommand(value = "stats wa", help = "Write Amplification. Ratio of how many records were upserted to how many "
+ @ShellMethod(key = "stats wa", value = "Write Amplification. Ratio of how many records were upserted to how many "
+ "records were actually written")
public String writeAmplificationStats(
- @CliOption(key = {"limit"}, help = "Limit commits", unspecifiedDefaultValue = "-1") final Integer limit,
- @CliOption(key = {"sortBy"}, help = "Sorting Field", unspecifiedDefaultValue = "") final String sortByField,
- @CliOption(key = {"desc"}, help = "Ordering", unspecifiedDefaultValue = "false") final boolean descending,
- @CliOption(key = {"headeronly"}, help = "Print Header Only",
- unspecifiedDefaultValue = "false") final boolean headerOnly)
+ @ShellOption(value = {"--limit"}, help = "Limit commits", defaultValue = "-1") final Integer limit,
+ @ShellOption(value = {"--sortBy"}, help = "Sorting Field", defaultValue = "") final String sortByField,
+ @ShellOption(value = {"--desc"}, help = "Ordering", defaultValue = "false") final boolean descending,
+ @ShellOption(value = {"--headeronly"}, help = "Print Header Only",
+ defaultValue = "false") final boolean headerOnly)
throws IOException {
long totalRecordsUpserted = 0;
@@ -105,15 +103,15 @@ public Comparable[] printFileSizeHistogram(String instantTime, Snapshot s) {
s.getMax(), s.size(), s.getStdDev()};
}
- @CliCommand(value = "stats filesizes", help = "File Sizes. Display summary stats on sizes of files")
+ @ShellMethod(key = "stats filesizes", value = "File Sizes. Display summary stats on sizes of files")
public String fileSizeStats(
- @CliOption(key = {"partitionPath"}, help = "regex to select files, eg: 2016/08/02",
- unspecifiedDefaultValue = "*/*/*") final String globRegex,
- @CliOption(key = {"limit"}, help = "Limit commits", unspecifiedDefaultValue = "-1") final Integer limit,
- @CliOption(key = {"sortBy"}, help = "Sorting Field", unspecifiedDefaultValue = "") final String sortByField,
- @CliOption(key = {"desc"}, help = "Ordering", unspecifiedDefaultValue = "false") final boolean descending,
- @CliOption(key = {"headeronly"}, help = "Print Header Only",
- unspecifiedDefaultValue = "false") final boolean headerOnly)
+ @ShellOption(value = {"--partitionPath"}, help = "regex to select files, eg: 2016/08/02",
+ defaultValue = "*/*/*") final String globRegex,
+ @ShellOption(value = {"--limit"}, help = "Limit commits", defaultValue = "-1") final Integer limit,
+ @ShellOption(value = {"--sortBy"}, help = "Sorting Field", defaultValue = "") final String sortByField,
+ @ShellOption(value = {"--desc"}, help = "Ordering", defaultValue = "false") final boolean descending,
+ @ShellOption(value = {"--headeronly"}, help = "Print Header Only",
+ defaultValue = "false") final boolean headerOnly)
throws IOException {
FileSystem fs = HoodieCLI.fs;
diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/TableCommand.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/TableCommand.java
index 9c947e4d407e3..b3dfaf5ab73bc 100644
--- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/TableCommand.java
+++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/TableCommand.java
@@ -18,45 +18,63 @@
package org.apache.hudi.cli.commands;
+import org.apache.avro.Schema;
+import org.apache.hadoop.fs.Path;
import org.apache.hudi.cli.HoodieCLI;
import org.apache.hudi.cli.HoodiePrintHelper;
+import org.apache.hudi.cli.HoodieTableHeaderFields;
import org.apache.hudi.cli.TableHeader;
import org.apache.hudi.common.fs.ConsistencyGuardConfig;
-import org.apache.hudi.common.model.HoodieTableType;
+import org.apache.hudi.common.table.HoodieTableConfig;
import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.TableSchemaResolver;
import org.apache.hudi.exception.TableNotFoundException;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+import org.springframework.shell.standard.ShellComponent;
+import org.springframework.shell.standard.ShellMethod;
+import org.springframework.shell.standard.ShellOption;
-import org.springframework.shell.core.CommandMarker;
-import org.springframework.shell.core.annotation.CliCommand;
-import org.springframework.shell.core.annotation.CliOption;
-import org.springframework.stereotype.Component;
-
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
import java.io.IOException;
+import java.io.OutputStream;
import java.util.ArrayList;
+import java.util.Arrays;
import java.util.HashMap;
import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.stream.Collectors;
+
+import static org.apache.hudi.common.table.HoodieTableMetaClient.METAFOLDER_NAME;
/**
* CLI command to display hudi table options.
*/
-@Component
-public class TableCommand implements CommandMarker {
+@ShellComponent
+public class TableCommand {
+
+ private static final Logger LOG = LogManager.getLogger(TableCommand.class);
static {
System.out.println("Table command getting loaded");
}
- @CliCommand(value = "connect", help = "Connect to a hoodie table")
+ @ShellMethod(key = "connect", value = "Connect to a hoodie table")
public String connect(
- @CliOption(key = {"path"}, mandatory = true, help = "Base Path of the table") final String path,
- @CliOption(key = {"layoutVersion"}, help = "Timeline Layout version") Integer layoutVersion,
- @CliOption(key = {"eventuallyConsistent"}, unspecifiedDefaultValue = "false",
+ @ShellOption(value = {"--path"}, help = "Base Path of the table") final String path,
+ @ShellOption(value = {"--layoutVersion"}, help = "Timeline Layout version", defaultValue = ShellOption.NULL) Integer layoutVersion,
+ @ShellOption(value = {"--eventuallyConsistent"}, defaultValue = "false",
help = "Enable eventual consistency") final boolean eventuallyConsistent,
- @CliOption(key = {"initialCheckIntervalMs"}, unspecifiedDefaultValue = "2000",
+ @ShellOption(value = {"--initialCheckIntervalMs"}, defaultValue = "2000",
help = "Initial wait time for eventual consistency") final Integer initialConsistencyIntervalMs,
- @CliOption(key = {"maxWaitIntervalMs"}, unspecifiedDefaultValue = "300000",
+ @ShellOption(value = {"--maxWaitIntervalMs"}, defaultValue = "300000",
help = "Max wait time for eventual consistency") final Integer maxConsistencyIntervalMs,
- @CliOption(key = {"maxCheckIntervalMs"}, unspecifiedDefaultValue = "7",
+ @ShellOption(value = {"--maxCheckIntervalMs"}, defaultValue = "7",
help = "Max checks for eventual consistency") final Integer maxConsistencyChecks)
throws IOException {
HoodieCLI
@@ -74,20 +92,22 @@ public String connect(
/**
* Create a Hoodie Table if it does not exist.
*
- * @param path Base Path
- * @param name Hoodie Table Name
+ * @param path Base Path
+ * @param name Hoodie Table Name
* @param tableTypeStr Hoodie Table Type
* @param payloadClass Payload Class
*/
- @CliCommand(value = "create", help = "Create a hoodie table if not present")
+ @ShellMethod(key = "create", value = "Create a hoodie table if not present")
public String createTable(
- @CliOption(key = {"path"}, mandatory = true, help = "Base Path of the table") final String path,
- @CliOption(key = {"tableName"}, mandatory = true, help = "Hoodie Table Name") final String name,
- @CliOption(key = {"tableType"}, unspecifiedDefaultValue = "COPY_ON_WRITE",
+ @ShellOption(value = {"--path"}, help = "Base Path of the table") final String path,
+ @ShellOption(value = {"--tableName"}, help = "Hoodie Table Name") final String name,
+ @ShellOption(value = {"--tableType"}, defaultValue = "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",
+ @ShellOption(value = {"--archiveLogFolder"}, help = "Folder Name for storing archived timeline",
+ defaultValue = ShellOption.NULL) String archiveFolder,
+ @ShellOption(value = {"--layoutVersion"}, help = "Specific Layout Version to use",
+ defaultValue = ShellOption.NULL) Integer layoutVersion,
+ @ShellOption(value = {"--payloadClass"}, defaultValue = "org.apache.hudi.common.model.HoodieAvroPayload",
help = "Payload Class") final String payloadClass) throws IOException {
boolean initialized = HoodieCLI.initConf();
@@ -95,7 +115,7 @@ public String createTable(
boolean existing = false;
try {
- new HoodieTableMetaClient(HoodieCLI.conf, path);
+ HoodieTableMetaClient.builder().setConf(HoodieCLI.conf).setBasePath(path).build();
existing = true;
} catch (TableNotFoundException dfe) {
// expected
@@ -106,10 +126,13 @@ public String createTable(
throw new IllegalStateException("Table already existing in path : " + path);
}
- final HoodieTableType tableType = HoodieTableType.valueOf(tableTypeStr);
- HoodieTableMetaClient.initTableType(HoodieCLI.conf, path, tableType, name, archiveFolder,
- payloadClass, layoutVersion);
-
+ HoodieTableMetaClient.withPropertyBuilder()
+ .setTableType(tableTypeStr)
+ .setTableName(name)
+ .setArchiveLogFolder(archiveFolder)
+ .setPayloadClassName(payloadClass)
+ .setTimelineLayoutVersion(layoutVersion)
+ .initTable(HoodieCLI.conf, path);
// Now connect to ensure loading works
return connect(path, layoutVersion, false, 0, 0, 0);
}
@@ -117,7 +140,7 @@ public String createTable(
/**
* Describes table properties.
*/
- @CliCommand(value = "desc", help = "Describe Hoodie Table properties")
+ @ShellMethod(key = "desc", value = "Describe Hoodie Table properties")
public String descTable() {
HoodieTableMetaClient client = HoodieCLI.getTableMetaClient();
TableHeader header = new TableHeader().addTableHeaderField("Property").addTableHeaderField("Value");
@@ -125,7 +148,7 @@ public String descTable() {
rows.add(new Comparable[] {"basePath", client.getBasePath()});
rows.add(new Comparable[] {"metaPath", client.getMetaPath()});
rows.add(new Comparable[] {"fileSystem", client.getFs().getScheme()});
- client.getTableConfig().getProps().entrySet().forEach(e -> {
+ client.getTableConfig().propsMap().entrySet().forEach(e -> {
rows.add(new Comparable[] {e.getKey(), e.getValue()});
});
return HoodiePrintHelper.print(header, new HashMap<>(), "", false, -1, false, rows);
@@ -134,10 +157,110 @@ public String descTable() {
/**
* Refresh table metadata.
*/
- @CliCommand(value = {"refresh", "metadata refresh", "commits refresh", "cleans refresh", "savepoints refresh"},
- help = "Refresh table metadata")
+ @ShellMethod(key = {"refresh", "metadata refresh", "commits refresh", "cleans refresh", "savepoints refresh"},
+ value = "Refresh table metadata")
public String refreshMetadata() {
HoodieCLI.refreshTableMetadata();
return "Metadata for table " + HoodieCLI.getTableMetaClient().getTableConfig().getTableName() + " refreshed.";
}
+
+ /**
+ * Fetches table schema in avro format.
+ */
+ @ShellMethod(key = "fetch table schema", value = "Fetches latest table schema")
+ public String fetchTableSchema(
+ @ShellOption(value = {"--outputFilePath"}, defaultValue = ShellOption.NULL,
+ help = "File path to write schema") final String outputFilePath) throws Exception {
+ HoodieTableMetaClient client = HoodieCLI.getTableMetaClient();
+ TableSchemaResolver tableSchemaResolver = new TableSchemaResolver(client);
+ Schema schema = tableSchemaResolver.getTableAvroSchema();
+ if (outputFilePath != null) {
+ LOG.info("Latest table schema : " + schema.toString(true));
+ writeToFile(outputFilePath, schema.toString(true));
+ return String.format("Latest table schema written to %s", outputFilePath);
+ } else {
+ return String.format("Latest table schema %s", schema.toString(true));
+ }
+ }
+
+ @ShellMethod(key = "table recover-configs", value = "Recover table configs, from update/delete that failed midway.")
+ public String recoverTableConfig() throws IOException {
+ HoodieCLI.refreshTableMetadata();
+ HoodieTableMetaClient client = HoodieCLI.getTableMetaClient();
+ Path metaPathDir = new Path(client.getBasePath(), METAFOLDER_NAME);
+ HoodieTableConfig.recover(client.getFs(), metaPathDir);
+ return descTable();
+ }
+
+ @ShellMethod(key = "table update-configs", value = "Update the table configs with configs with provided file.")
+ public String updateTableConfig(
+ @ShellOption(value = {"--props-file"}, help = "Path to a properties file on local filesystem")
+ final String updatePropsFilePath) throws IOException {
+ HoodieTableMetaClient client = HoodieCLI.getTableMetaClient();
+ Map oldProps = client.getTableConfig().propsMap();
+
+ Properties updatedProps = new Properties();
+ updatedProps.load(new FileInputStream(updatePropsFilePath));
+ Path metaPathDir = new Path(client.getBasePath(), METAFOLDER_NAME);
+ HoodieTableConfig.update(client.getFs(), metaPathDir, updatedProps);
+
+ HoodieCLI.refreshTableMetadata();
+ Map newProps = HoodieCLI.getTableMetaClient().getTableConfig().propsMap();
+ return renderOldNewProps(newProps, oldProps);
+ }
+
+ @ShellMethod(key = "table delete-configs", value = "Delete the supplied table configs from the table.")
+ public String deleteTableConfig(
+ @ShellOption(value = {"--comma-separated-configs"},
+ help = "Comma separated list of configs to delete.") final String csConfigs) {
+ HoodieTableMetaClient client = HoodieCLI.getTableMetaClient();
+ Map oldProps = client.getTableConfig().propsMap();
+
+ Set deleteConfigs = Arrays.stream(csConfigs.split(",")).collect(Collectors.toSet());
+ Path metaPathDir = new Path(client.getBasePath(), METAFOLDER_NAME);
+ HoodieTableConfig.delete(client.getFs(), metaPathDir, deleteConfigs);
+
+ HoodieCLI.refreshTableMetadata();
+ Map newProps = HoodieCLI.getTableMetaClient().getTableConfig().propsMap();
+ return renderOldNewProps(newProps, oldProps);
+ }
+
+ private static String renderOldNewProps(Map newProps, Map oldProps) {
+ TreeSet allPropKeys = new TreeSet<>();
+ allPropKeys.addAll(newProps.keySet().stream().map(Object::toString).collect(Collectors.toSet()));
+ allPropKeys.addAll(oldProps.keySet());
+
+ String[][] rows = new String[allPropKeys.size()][];
+ int ind = 0;
+ for (String propKey : allPropKeys) {
+ String[] row = new String[] {
+ propKey,
+ oldProps.getOrDefault(propKey, "null"),
+ newProps.getOrDefault(propKey, "null")
+ };
+ rows[ind++] = row;
+ }
+ return HoodiePrintHelper.print(new String[] {HoodieTableHeaderFields.HEADER_HOODIE_PROPERTY,
+ HoodieTableHeaderFields.HEADER_OLD_VALUE, HoodieTableHeaderFields.HEADER_NEW_VALUE}, rows);
+ }
+
+ /**
+ * Use Streams when you are dealing with raw data.
+ *
+ * @param filePath output file path.
+ * @param data to be written to file.
+ */
+ private static void writeToFile(String filePath, String data) throws IOException {
+ File outFile = new File(filePath);
+ if (outFile.exists()) {
+ outFile.delete();
+ }
+ OutputStream os = null;
+ try {
+ os = new FileOutputStream(outFile);
+ os.write(data.getBytes(), 0, data.length());
+ } finally {
+ os.close();
+ }
+ }
}
diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/TempViewCommand.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/TempViewCommand.java
index 975e89fc772a1..3f88532b568ec 100644
--- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/TempViewCommand.java
+++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/TempViewCommand.java
@@ -19,26 +19,24 @@
package org.apache.hudi.cli.commands;
import org.apache.hudi.cli.HoodieCLI;
-
import org.apache.hudi.exception.HoodieException;
-import org.springframework.shell.core.CommandMarker;
-import org.springframework.shell.core.annotation.CliCommand;
-import org.springframework.shell.core.annotation.CliOption;
-import org.springframework.stereotype.Component;
+import org.springframework.shell.standard.ShellComponent;
+import org.springframework.shell.standard.ShellMethod;
+import org.springframework.shell.standard.ShellOption;
/**
* CLI command to query/delete temp views.
*/
-@Component
-public class TempViewCommand implements CommandMarker {
+@ShellComponent
+public class TempViewCommand {
public static final String QUERY_SUCCESS = "Query ran successfully!";
public static final String QUERY_FAIL = "Query ran failed!";
public static final String SHOW_SUCCESS = "Show all views name successfully!";
- @CliCommand(value = {"temp_query", "temp query"}, help = "query against created temp view")
+ @ShellMethod(key = {"temp_query", "temp query"}, value = "query against created temp view")
public String query(
- @CliOption(key = {"sql"}, mandatory = true, help = "select query to run against view") final String sql) {
+ @ShellOption(value = {"--sql"}, help = "select query to run against view") final String sql) {
try {
HoodieCLI.getTempViewProvider().runQuery(sql);
@@ -49,7 +47,7 @@ public String query(
}
- @CliCommand(value = {"temps_show", "temps show"}, help = "Show all views name")
+ @ShellMethod(key = {"temps_show", "temps show"}, value = "Show all views name")
public String showAll() {
try {
@@ -60,9 +58,9 @@ public String showAll() {
}
}
- @CliCommand(value = {"temp_delete", "temp delete"}, help = "Delete view name")
+ @ShellMethod(key = {"temp_delete", "temp delete"}, value = "Delete view name")
public String delete(
- @CliOption(key = {"view"}, mandatory = true, help = "view name") final String tableName) {
+ @ShellOption(value = {"--view"}, help = "view name") final String tableName) {
try {
HoodieCLI.getTempViewProvider().deleteTable(tableName);
diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/TimelineCommand.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/TimelineCommand.java
new file mode 100644
index 0000000000000..bf7e5397cab93
--- /dev/null
+++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/TimelineCommand.java
@@ -0,0 +1,408 @@
+/*
+ * 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.cli.commands;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hudi.avro.model.HoodieRollbackMetadata;
+import org.apache.hudi.avro.model.HoodieRollbackPlan;
+import org.apache.hudi.cli.HoodieCLI;
+import org.apache.hudi.cli.HoodiePrintHelper;
+import org.apache.hudi.cli.HoodieTableHeaderFields;
+import org.apache.hudi.cli.TableHeader;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.common.table.timeline.HoodieTimeline;
+import org.apache.hudi.common.table.timeline.TimelineMetadataUtils;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.metadata.HoodieTableMetadata;
+import org.apache.logging.log4j.Logger;
+import org.apache.logging.log4j.LogManager;
+import org.springframework.shell.standard.ShellComponent;
+import org.springframework.shell.standard.ShellMethod;
+import org.springframework.shell.standard.ShellOption;
+
+import java.io.IOException;
+import java.text.SimpleDateFormat;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * CLI command to display timeline options.
+ */
+@ShellComponent
+public class TimelineCommand {
+
+ private static final Logger LOG = LogManager.getLogger(TimelineCommand.class);
+ private static final SimpleDateFormat DATE_FORMAT_DEFAULT = new SimpleDateFormat("MM-dd HH:mm");
+ private static final SimpleDateFormat DATE_FORMAT_SECONDS = new SimpleDateFormat("MM-dd HH:mm:ss");
+
+ @ShellMethod(key = "timeline show active", value = "List all instants in active timeline")
+ public String showActive(
+ @ShellOption(value = {"--limit"}, help = "Limit #rows to be displayed", defaultValue = "10") Integer limit,
+ @ShellOption(value = {"--sortBy"}, help = "Sorting Field", defaultValue = "") final String sortByField,
+ @ShellOption(value = {"--desc"}, help = "Ordering", defaultValue = "false") final boolean descending,
+ @ShellOption(value = {"--headeronly"}, help = "Print Header Only",
+ defaultValue = "false") final boolean headerOnly,
+ @ShellOption(value = {"--with-metadata-table"}, help = "Show metadata table timeline together with data table",
+ defaultValue = "false") final boolean withMetadataTable,
+ @ShellOption(value = {"--show-rollback-info"}, help = "Show instant to rollback for rollbacks",
+ defaultValue = "false") final boolean showRollbackInfo,
+ @ShellOption(value = {"--show-time-seconds"}, help = "Show seconds in instant file modification time",
+ defaultValue = "false") final boolean showTimeSeconds) {
+ HoodieTableMetaClient metaClient = HoodieCLI.getTableMetaClient();
+ try {
+ if (withMetadataTable) {
+ HoodieTableMetaClient mtMetaClient = getMetadataTableMetaClient(metaClient);
+ return printTimelineInfoWithMetadataTable(
+ metaClient.getActiveTimeline(), mtMetaClient.getActiveTimeline(),
+ getInstantInfoFromTimeline(metaClient.getFs(), metaClient.getMetaPath()),
+ getInstantInfoFromTimeline(mtMetaClient.getFs(), mtMetaClient.getMetaPath()),
+ limit, sortByField, descending, headerOnly, true, showTimeSeconds, showRollbackInfo);
+ }
+ return printTimelineInfo(
+ metaClient.getActiveTimeline(),
+ getInstantInfoFromTimeline(metaClient.getFs(), metaClient.getMetaPath()),
+ limit, sortByField, descending, headerOnly, true, showTimeSeconds, showRollbackInfo);
+ } catch (IOException e) {
+ e.printStackTrace();
+ return e.getMessage();
+ }
+ }
+
+ @ShellMethod(key = "timeline show incomplete", value = "List all incomplete instants in active timeline")
+ public String showIncomplete(
+ @ShellOption(value = {"--limit"}, help = "Limit #rows to be displayed", defaultValue = "10") Integer limit,
+ @ShellOption(value = {"--sortBy"}, help = "Sorting Field", defaultValue = "") final String sortByField,
+ @ShellOption(value = {"--desc"}, help = "Ordering", defaultValue = "false") final boolean descending,
+ @ShellOption(value = {"--headeronly"}, help = "Print Header Only",
+ defaultValue = "false") final boolean headerOnly,
+ @ShellOption(value = {"--show-rollback-info"}, help = "Show instant to rollback for rollbacks",
+ defaultValue = "false") final boolean showRollbackInfo,
+ @ShellOption(value = {"--show-time-seconds"}, help = "Show seconds in instant file modification time",
+ defaultValue = "false") final boolean showTimeSeconds) {
+ HoodieTableMetaClient metaClient = HoodieCLI.getTableMetaClient();
+ try {
+ return printTimelineInfo(
+ metaClient.getActiveTimeline().filterInflightsAndRequested(),
+ getInstantInfoFromTimeline(metaClient.getFs(), metaClient.getMetaPath()),
+ limit, sortByField, descending, headerOnly, true, showTimeSeconds, showRollbackInfo);
+ } catch (IOException e) {
+ e.printStackTrace();
+ return e.getMessage();
+ }
+ }
+
+ @ShellMethod(key = "metadata timeline show active",
+ value = "List all instants in active timeline of metadata table")
+ public String metadataShowActive(
+ @ShellOption(value = {"--limit"}, help = "Limit #rows to be displayed", defaultValue = "10") Integer limit,
+ @ShellOption(value = {"--sortBy"}, help = "Sorting Field", defaultValue = "") final String sortByField,
+ @ShellOption(value = {"--desc"}, help = "Ordering", defaultValue = "false") final boolean descending,
+ @ShellOption(value = {"--headeronly"}, help = "Print Header Only",
+ defaultValue = "false") final boolean headerOnly,
+ @ShellOption(value = {"--show-time-seconds"}, help = "Show seconds in instant file modification time",
+ defaultValue = "false") final boolean showTimeSeconds) {
+ HoodieTableMetaClient metaClient = getMetadataTableMetaClient(HoodieCLI.getTableMetaClient());
+ try {
+ return printTimelineInfo(
+ metaClient.getActiveTimeline(),
+ getInstantInfoFromTimeline(metaClient.getFs(), metaClient.getMetaPath()),
+ limit, sortByField, descending, headerOnly, true, showTimeSeconds, false);
+ } catch (IOException e) {
+ e.printStackTrace();
+ return e.getMessage();
+ }
+ }
+
+ @ShellMethod(key = "metadata timeline show incomplete",
+ value = "List all incomplete instants in active timeline of metadata table")
+ public String metadataShowIncomplete(
+ @ShellOption(value = {"--limit"}, help = "Limit #rows to be displayed", defaultValue = "10") Integer limit,
+ @ShellOption(value = {"--sortBy"}, help = "Sorting Field", defaultValue = "") final String sortByField,
+ @ShellOption(value = {"--desc"}, help = "Ordering", defaultValue = "false") final boolean descending,
+ @ShellOption(value = {"--headeronly"}, help = "Print Header Only",
+ defaultValue = "false") final boolean headerOnly,
+ @ShellOption(value = {"--show-time-seconds"}, help = "Show seconds in instant file modification time",
+ defaultValue = "false") final boolean showTimeSeconds) {
+ HoodieTableMetaClient metaClient = getMetadataTableMetaClient(HoodieCLI.getTableMetaClient());
+ try {
+ return printTimelineInfo(
+ metaClient.getActiveTimeline().filterInflightsAndRequested(),
+ getInstantInfoFromTimeline(metaClient.getFs(), metaClient.getMetaPath()),
+ limit, sortByField, descending, headerOnly, true, showTimeSeconds, false);
+ } catch (IOException e) {
+ e.printStackTrace();
+ return e.getMessage();
+ }
+ }
+
+ private HoodieTableMetaClient getMetadataTableMetaClient(HoodieTableMetaClient metaClient) {
+ return HoodieTableMetaClient.builder().setConf(HoodieCLI.conf)
+ .setBasePath(HoodieTableMetadata.getMetadataTableBasePath(metaClient.getBasePath()))
+ .setLoadActiveTimelineOnLoad(false)
+ .setConsistencyGuardConfig(HoodieCLI.consistencyGuardConfig)
+ .build();
+ }
+
+ private Map> getInstantInfoFromTimeline(
+ FileSystem fs, String metaPath) throws IOException {
+ Map> instantMap = new HashMap<>();
+ Stream instantStream = Arrays.stream(
+ HoodieTableMetaClient.scanFiles(fs, new Path(metaPath), path -> {
+ // Include only the meta files with extensions that needs to be included
+ String extension = HoodieInstant.getTimelineFileExtension(path.getName());
+ return HoodieActiveTimeline.VALID_EXTENSIONS_IN_ACTIVE_TIMELINE.contains(extension);
+ })).map(HoodieInstantWithModTime::new);
+ instantStream.forEach(instant -> {
+ instantMap.computeIfAbsent(instant.getTimestamp(), t -> new HashMap<>())
+ .put(instant.getState(), instant);
+ });
+ return instantMap;
+ }
+
+ private String getFormattedDate(
+ String instantTimestamp, HoodieInstant.State state,
+ Map> instantInfoMap,
+ boolean showTimeSeconds) {
+ Long timeMs = null;
+ Map mapping = instantInfoMap.get(instantTimestamp);
+ if (mapping != null && mapping.containsKey(state)) {
+ timeMs = mapping.get(state).getModificationTime();
+ }
+ SimpleDateFormat sdf = showTimeSeconds ? DATE_FORMAT_SECONDS : DATE_FORMAT_DEFAULT;
+ return timeMs != null ? sdf.format(new Date(timeMs)) : "-";
+ }
+
+ private String printTimelineInfo(
+ HoodieTimeline timeline,
+ Map> instantInfoMap,
+ Integer limit, String sortByField, boolean descending, boolean headerOnly, boolean withRowNo,
+ boolean showTimeSeconds, boolean showRollbackInfo) {
+ Map> rollbackInfo = getRolledBackInstantInfo(timeline);
+ final List rows = timeline.getInstants().map(instant -> {
+ int numColumns = showRollbackInfo ? 7 : 6;
+ Comparable[] row = new Comparable[numColumns];
+ String instantTimestamp = instant.getTimestamp();
+ row[0] = instantTimestamp;
+ row[1] = instant.getAction();
+ row[2] = instant.getState();
+ if (showRollbackInfo) {
+ if (HoodieTimeline.ROLLBACK_ACTION.equalsIgnoreCase(instant.getAction())) {
+ row[3] = "Rolls back\n" + getInstantToRollback(timeline, instant);
+ } else {
+ if (rollbackInfo.containsKey(instantTimestamp)) {
+ row[3] = "Rolled back by\n" + String.join(",\n", rollbackInfo.get(instantTimestamp));
+ } else {
+ row[3] = "-";
+ }
+ }
+ }
+ row[numColumns - 3] = getFormattedDate(
+ instantTimestamp, HoodieInstant.State.REQUESTED, instantInfoMap, showTimeSeconds);
+ row[numColumns - 2] = getFormattedDate(
+ instantTimestamp, HoodieInstant.State.INFLIGHT, instantInfoMap, showTimeSeconds);
+ row[numColumns - 1] = getFormattedDate(
+ instantTimestamp, HoodieInstant.State.COMPLETED, instantInfoMap, showTimeSeconds);
+ return row;
+ }).collect(Collectors.toList());
+ TableHeader header = new TableHeader()
+ .addTableHeaderField(HoodieTableHeaderFields.HEADER_INSTANT)
+ .addTableHeaderField(HoodieTableHeaderFields.HEADER_ACTION)
+ .addTableHeaderField(HoodieTableHeaderFields.HEADER_STATE);
+ if (showRollbackInfo) {
+ header.addTableHeaderField(HoodieTableHeaderFields.HEADER_ROLLBACK_INFO);
+ }
+ header.addTableHeaderField(HoodieTableHeaderFields.HEADER_REQUESTED_TIME)
+ .addTableHeaderField(HoodieTableHeaderFields.HEADER_INFLIGHT_TIME)
+ .addTableHeaderField(HoodieTableHeaderFields.HEADER_COMPLETED_TIME);
+ return HoodiePrintHelper.print(
+ header, new HashMap<>(), withRowNo, sortByField, descending, limit, headerOnly, rows);
+ }
+
+ private String printTimelineInfoWithMetadataTable(
+ HoodieTimeline dtTimeline, HoodieTimeline mtTimeline,
+ Map> dtInstantInfoMap,
+ Map> mtInstantInfoMap,
+ Integer limit, String sortByField, boolean descending, boolean headerOnly, boolean withRowNo,
+ boolean showTimeSeconds, boolean showRollbackInfo) {
+ Set instantTimeSet = new HashSet(dtInstantInfoMap.keySet());
+ instantTimeSet.addAll(mtInstantInfoMap.keySet());
+ List instantTimeList = instantTimeSet.stream()
+ .sorted(new HoodieInstantTimeComparator()).collect(Collectors.toList());
+ Map> dtRollbackInfo = getRolledBackInstantInfo(dtTimeline);
+
+ final List rows = instantTimeList.stream().map(instantTimestamp -> {
+ int numColumns = showRollbackInfo ? 12 : 11;
+ Option dtInstant = getInstant(dtTimeline, instantTimestamp);
+ Option mtInstant = getInstant(mtTimeline, instantTimestamp);
+ Comparable[] row = new Comparable[numColumns];
+ row[0] = instantTimestamp;
+ row[1] = dtInstant.isPresent() ? dtInstant.get().getAction() : "-";
+ row[2] = dtInstant.isPresent() ? dtInstant.get().getState() : "-";
+ if (showRollbackInfo) {
+ if (dtInstant.isPresent()
+ && HoodieTimeline.ROLLBACK_ACTION.equalsIgnoreCase(dtInstant.get().getAction())) {
+ row[3] = "Rolls back\n" + getInstantToRollback(dtTimeline, dtInstant.get());
+ } else {
+ if (dtRollbackInfo.containsKey(instantTimestamp)) {
+ row[3] = "Rolled back by\n" + String.join(",\n", dtRollbackInfo.get(instantTimestamp));
+ } else {
+ row[3] = "-";
+ }
+ }
+ }
+ row[numColumns - 8] = getFormattedDate(
+ instantTimestamp, HoodieInstant.State.REQUESTED, dtInstantInfoMap, showTimeSeconds);
+ row[numColumns - 7] = getFormattedDate(
+ instantTimestamp, HoodieInstant.State.INFLIGHT, dtInstantInfoMap, showTimeSeconds);
+ row[numColumns - 6] = getFormattedDate(
+ instantTimestamp, HoodieInstant.State.COMPLETED, dtInstantInfoMap, showTimeSeconds);
+ row[numColumns - 5] = mtInstant.isPresent() ? mtInstant.get().getAction() : "-";
+ row[numColumns - 4] = mtInstant.isPresent() ? mtInstant.get().getState() : "-";
+ row[numColumns - 3] = getFormattedDate(
+ instantTimestamp, HoodieInstant.State.REQUESTED, mtInstantInfoMap, showTimeSeconds);
+ row[numColumns - 2] = getFormattedDate(
+ instantTimestamp, HoodieInstant.State.INFLIGHT, mtInstantInfoMap, showTimeSeconds);
+ row[numColumns - 1] = getFormattedDate(
+ instantTimestamp, HoodieInstant.State.COMPLETED, mtInstantInfoMap, showTimeSeconds);
+ return row;
+ }).collect(Collectors.toList());
+ TableHeader header = new TableHeader()
+ .addTableHeaderField(HoodieTableHeaderFields.HEADER_INSTANT)
+ .addTableHeaderField(HoodieTableHeaderFields.HEADER_ACTION)
+ .addTableHeaderField(HoodieTableHeaderFields.HEADER_STATE);
+ if (showRollbackInfo) {
+ header.addTableHeaderField(HoodieTableHeaderFields.HEADER_ROLLBACK_INFO);
+ }
+ header.addTableHeaderField(HoodieTableHeaderFields.HEADER_REQUESTED_TIME)
+ .addTableHeaderField(HoodieTableHeaderFields.HEADER_INFLIGHT_TIME)
+ .addTableHeaderField(HoodieTableHeaderFields.HEADER_COMPLETED_TIME)
+ .addTableHeaderField(HoodieTableHeaderFields.HEADER_MT_ACTION)
+ .addTableHeaderField(HoodieTableHeaderFields.HEADER_MT_STATE)
+ .addTableHeaderField(HoodieTableHeaderFields.HEADER_MT_REQUESTED_TIME)
+ .addTableHeaderField(HoodieTableHeaderFields.HEADER_MT_INFLIGHT_TIME)
+ .addTableHeaderField(HoodieTableHeaderFields.HEADER_MT_COMPLETED_TIME);
+ return HoodiePrintHelper.print(
+ header, new HashMap<>(), withRowNo, sortByField, descending, limit, headerOnly, rows);
+ }
+
+ private Option getInstant(HoodieTimeline timeline, String instantTimestamp) {
+ return timeline.filter(instant -> instant.getTimestamp().equals(instantTimestamp)).firstInstant();
+ }
+
+ private String getInstantToRollback(HoodieTimeline timeline, HoodieInstant instant) {
+ try {
+ if (instant.isInflight()) {
+ HoodieInstant instantToUse = new HoodieInstant(
+ HoodieInstant.State.REQUESTED, instant.getAction(), instant.getTimestamp());
+ HoodieRollbackPlan metadata = TimelineMetadataUtils
+ .deserializeAvroMetadata(timeline.getInstantDetails(instantToUse).get(), HoodieRollbackPlan.class);
+ return metadata.getInstantToRollback().getCommitTime();
+ } else {
+ HoodieRollbackMetadata metadata = TimelineMetadataUtils
+ .deserializeAvroMetadata(timeline.getInstantDetails(instant).get(), HoodieRollbackMetadata.class);
+ return String.join(",", metadata.getCommitsRollback());
+ }
+ } catch (IOException e) {
+ LOG.error(String.format("Error reading rollback info of %s", instant));
+ e.printStackTrace();
+ return "-";
+ }
+ }
+
+ private Map> getRolledBackInstantInfo(HoodieTimeline timeline) {
+ // Instant rolled back or to roll back -> rollback instants
+ Map> rollbackInfoMap = new HashMap<>();
+ List rollbackInstants = timeline.filter(instant ->
+ HoodieTimeline.ROLLBACK_ACTION.equalsIgnoreCase(instant.getAction()))
+ .getInstants().collect(Collectors.toList());
+ rollbackInstants.forEach(rollbackInstant -> {
+ try {
+ if (rollbackInstant.isInflight()) {
+ HoodieInstant instantToUse = new HoodieInstant(
+ HoodieInstant.State.REQUESTED, rollbackInstant.getAction(), rollbackInstant.getTimestamp());
+ HoodieRollbackPlan metadata = TimelineMetadataUtils
+ .deserializeAvroMetadata(timeline.getInstantDetails(instantToUse).get(), HoodieRollbackPlan.class);
+ rollbackInfoMap.computeIfAbsent(metadata.getInstantToRollback().getCommitTime(), k -> new ArrayList<>())
+ .add(rollbackInstant.getTimestamp());
+ } else {
+ HoodieRollbackMetadata metadata = TimelineMetadataUtils
+ .deserializeAvroMetadata(timeline.getInstantDetails(rollbackInstant).get(), HoodieRollbackMetadata.class);
+ metadata.getCommitsRollback().forEach(instant -> {
+ rollbackInfoMap.computeIfAbsent(instant, k -> new ArrayList<>())
+ .add(rollbackInstant.getTimestamp());
+ });
+ }
+ } catch (IOException e) {
+ LOG.error(String.format("Error reading rollback info of %s", rollbackInstant));
+ e.printStackTrace();
+ }
+ });
+ return rollbackInfoMap;
+ }
+
+ static class HoodieInstantWithModTime extends HoodieInstant {
+
+ private final long modificationTimeMs;
+
+ public HoodieInstantWithModTime(FileStatus fileStatus) {
+ super(fileStatus);
+ this.modificationTimeMs = fileStatus.getModificationTime();
+ }
+
+ public long getModificationTime() {
+ return modificationTimeMs;
+ }
+ }
+
+ static class HoodieInstantTimeComparator implements Comparator {
+ @Override
+ public int compare(String o1, String o2) {
+ // For metadata table, the compaction instant time is "012345001" while the delta commit
+ // later is "012345", i.e., the compaction instant time has trailing "001". In the
+ // actual event sequence, metadata table compaction happens before the corresponding
+ // delta commit. For better visualization, we put "012345001" before "012345"
+ // when sorting in ascending order.
+ if (o1.length() != o2.length()) {
+ // o1 is longer than o2
+ if (o1.length() - o2.length() == 3 && o1.endsWith("001") && o1.startsWith(o2)) {
+ return -1;
+ }
+ // o1 is shorter than o2
+ if (o2.length() - o1.length() == 3 && o2.endsWith("001") && o2.startsWith(o1)) {
+ return 1;
+ }
+ }
+ return o1.compareTo(o2);
+ }
+ }
+}
diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/UpgradeOrDowngradeCommand.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/UpgradeOrDowngradeCommand.java
index deb9e0727171b..5561723d7a57a 100644
--- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/UpgradeOrDowngradeCommand.java
+++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/UpgradeOrDowngradeCommand.java
@@ -23,59 +23,79 @@
import org.apache.hudi.cli.utils.InputStreamConsumer;
import org.apache.hudi.cli.utils.SparkUtil;
import org.apache.hudi.common.table.HoodieTableMetaClient;
-
+import org.apache.hudi.common.table.HoodieTableVersion;
+import org.apache.hudi.common.util.StringUtils;
import org.apache.spark.launcher.SparkLauncher;
-import org.springframework.shell.core.CommandMarker;
-import org.springframework.shell.core.annotation.CliCommand;
-import org.springframework.shell.core.annotation.CliOption;
+import org.springframework.shell.standard.ShellComponent;
+import org.springframework.shell.standard.ShellMethod;
+import org.springframework.shell.standard.ShellOption;
/**
- * CLI command to assist in upgrading/downgrading Hoodie dataset to a different version.
+ * CLI command to assist in upgrading/downgrading Hoodie table to a different version.
*/
-public class UpgradeOrDowngradeCommand implements CommandMarker {
+@ShellComponent
+public class UpgradeOrDowngradeCommand {
- @CliCommand(value = "upgrade hoodie dataset ", help = "Upgrades hoodie dataset")
- public String upgradeHoodieDataset(
- @CliOption(key = {"toVersion"}, help = "To version of Hoodie dataset to be upgraded/downgraded to", unspecifiedDefaultValue = "") final String toVersion,
- @CliOption(key = {"sparkProperties"}, help = "Spark Properties File Path") final String sparkPropertiesPath,
- @CliOption(key = "sparkMaster", unspecifiedDefaultValue = "", help = "Spark Master") String master,
- @CliOption(key = "sparkMemory", unspecifiedDefaultValue = "4G",
+ @ShellMethod(key = "upgrade table", value = "Upgrades a table")
+ public String upgradeHoodieTable(
+ @ShellOption(value = {"--toVersion"}, help = "To version of Hoodie table to be upgraded/downgraded to", defaultValue = "") final String toVersion,
+ @ShellOption(value = {"--sparkProperties"}, help = "Spark Properties File Path",
+ defaultValue = "") final String sparkPropertiesPath,
+ @ShellOption(value = "--sparkMaster", defaultValue = "", help = "Spark Master") String master,
+ @ShellOption(value = "--sparkMemory", defaultValue = "4G",
help = "Spark executor memory") final String sparkMemory)
throws Exception {
HoodieTableMetaClient metaClient = HoodieCLI.getTableMetaClient();
SparkLauncher sparkLauncher = SparkUtil.initLauncher(sparkPropertiesPath);
- sparkLauncher.addAppArgs(SparkCommand.UPGRADE.toString(), master, sparkMemory, metaClient.getBasePath(), toVersion);
+ String toVersionName = getHoodieTableVersionName(toVersion, true);
+ sparkLauncher.addAppArgs(SparkCommand.UPGRADE.toString(), master, sparkMemory, metaClient.getBasePath(), toVersionName);
Process process = sparkLauncher.launch();
InputStreamConsumer.captureOutput(process);
int exitCode = process.waitFor();
HoodieCLI.refreshTableMetadata();
if (exitCode != 0) {
- return String.format("Failed: Could not Upgrade/Downgrade Hoodie dataset to \"%s\".", toVersion);
+ return String.format("Failed: Could not Upgrade/Downgrade Hoodie table to \"%s\".", toVersionName);
}
- return String.format("Hoodie dataset upgraded/downgraded to ", toVersion);
+ return String.format("Hoodie table upgraded/downgraded to %s", toVersionName);
}
- @CliCommand(value = "downgrade hoodie dataset ", help = "Upgrades hoodie dataset")
- public String downgradeHoodieDataset(
- @CliOption(key = {"toVersion"}, help = "To version of Hoodie dataset to be upgraded/downgraded to", unspecifiedDefaultValue = "") final String toVersion,
- @CliOption(key = {"sparkProperties"}, help = "Spark Properties File Path") final String sparkPropertiesPath,
- @CliOption(key = "sparkMaster", unspecifiedDefaultValue = "", help = "Spark Master") String master,
- @CliOption(key = "sparkMemory", unspecifiedDefaultValue = "4G",
+ @ShellMethod(key = "downgrade table", value = "Downgrades a table")
+ public String downgradeHoodieTable(
+ @ShellOption(value = {"--toVersion"}, help = "To version of Hoodie table to be upgraded/downgraded to", defaultValue = "") final String toVersion,
+ @ShellOption(value = {"--sparkProperties"}, help = "Spark Properties File Path",
+ defaultValue = "") final String sparkPropertiesPath,
+ @ShellOption(value = "--sparkMaster", defaultValue = "", help = "Spark Master") String master,
+ @ShellOption(value = "--sparkMemory", defaultValue = "4G",
help = "Spark executor memory") final String sparkMemory)
throws Exception {
HoodieTableMetaClient metaClient = HoodieCLI.getTableMetaClient();
SparkLauncher sparkLauncher = SparkUtil.initLauncher(sparkPropertiesPath);
- sparkLauncher.addAppArgs(SparkCommand.DOWNGRADE.toString(), master, sparkMemory, metaClient.getBasePath(), toVersion);
+ String toVersionName = getHoodieTableVersionName(toVersion, false);
+ sparkLauncher.addAppArgs(SparkCommand.DOWNGRADE.toString(), master, sparkMemory, metaClient.getBasePath(), toVersionName);
Process process = sparkLauncher.launch();
InputStreamConsumer.captureOutput(process);
int exitCode = process.waitFor();
HoodieCLI.refreshTableMetadata();
if (exitCode != 0) {
- return String.format("Failed: Could not Upgrade/Downgrade Hoodie dataset to \"%s\".", toVersion);
+ return String.format("Failed: Could not Upgrade/Downgrade Hoodie table to \"%s\".", toVersionName);
+ }
+ return String.format("Hoodie table upgraded/downgraded to %s", toVersionName);
+ }
+
+ static String getHoodieTableVersionName(String versionOption, boolean overrideWithDefault) {
+ if (StringUtils.isNullOrEmpty(versionOption) && overrideWithDefault) {
+ return HoodieTableVersion.current().name();
+ }
+
+ try {
+ int versionCode = Integer.parseInt(versionOption);
+ return HoodieTableVersion.versionFromCode(versionCode).name();
+ } catch (NumberFormatException e) {
+ // The version option from the CLI is not a number, returns the original String
+ return versionOption;
}
- return String.format("Hoodie dataset upgraded/downgraded to ", toVersion);
}
}
diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/UtilsCommand.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/UtilsCommand.java
index 677cb7ffce2bd..2861b05b44d09 100644
--- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/UtilsCommand.java
+++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/UtilsCommand.java
@@ -19,19 +19,18 @@
package org.apache.hudi.cli.commands;
import org.apache.hudi.common.util.StringUtils;
-import org.springframework.shell.core.CommandMarker;
-import org.springframework.shell.core.annotation.CliCommand;
-import org.springframework.shell.core.annotation.CliOption;
-import org.springframework.stereotype.Component;
+import org.springframework.shell.standard.ShellComponent;
+import org.springframework.shell.standard.ShellMethod;
+import org.springframework.shell.standard.ShellOption;
/**
* CLI command to display utils.
*/
-@Component
-public class UtilsCommand implements CommandMarker {
+@ShellComponent
+public class UtilsCommand {
- @CliCommand(value = "utils loadClass", help = "Load a class")
- public String loadClass(@CliOption(key = {"class"}, help = "Check mode") final String clazz) {
+ @ShellMethod(key = "utils loadClass", value = "Load a class")
+ public String loadClass(@ShellOption(value = {"--class"}, help = "Check mode") final String clazz) {
if (StringUtils.isNullOrEmpty(clazz)) {
return "Class to be loaded can not be null!";
}
diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/utils/CommitUtil.java b/hudi-cli/src/main/java/org/apache/hudi/cli/utils/CommitUtil.java
index 5a1c457b10ef1..21910fd956dfe 100644
--- a/hudi-cli/src/main/java/org/apache/hudi/cli/utils/CommitUtil.java
+++ b/hudi-cli/src/main/java/org/apache/hudi/cli/utils/CommitUtil.java
@@ -25,9 +25,6 @@
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import java.io.IOException;
-import java.text.ParseException;
-import java.time.Instant;
-import java.time.ZoneId;
import java.time.ZonedDateTime;
import java.util.Date;
import java.util.List;
@@ -37,9 +34,9 @@
*/
public class CommitUtil {
- public static long countNewRecords(HoodieTableMetaClient target, List commitsToCatchup) throws IOException {
+ public static long countNewRecords(HoodieTableMetaClient metaClient, List commitsToCatchup) throws IOException {
long totalNew = 0;
- HoodieTimeline timeline = target.reloadActiveTimeline().getCommitTimeline().filterCompletedInstants();
+ HoodieTimeline timeline = metaClient.reloadActiveTimeline().getCommitTimeline().filterCompletedInstants();
for (String commit : commitsToCatchup) {
HoodieCommitMetadata c = HoodieCommitMetadata.fromBytes(
timeline.getInstantDetails(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commit)).get(),
@@ -51,18 +48,6 @@ public static long countNewRecords(HoodieTableMetaClient target, List co
public static String getTimeDaysAgo(int numberOfDays) {
Date date = Date.from(ZonedDateTime.now().minusDays(numberOfDays).toInstant());
- return HoodieActiveTimeline.COMMIT_FORMATTER.format(date);
- }
-
- /**
- * Add hours to specified time. If hours <0, this acts as remove hours.
- * example, say compactionCommitTime: "20200202020000"
- * a) hours: +1, returns 20200202030000
- * b) hours: -1, returns 20200202010000
- */
- public static String addHours(String compactionCommitTime, int hours) throws ParseException {
- Instant instant = HoodieActiveTimeline.COMMIT_FORMATTER.parse(compactionCommitTime).toInstant();
- ZonedDateTime commitDateTime = ZonedDateTime.ofInstant(instant, ZoneId.systemDefault());
- return HoodieActiveTimeline.COMMIT_FORMATTER.format(Date.from(commitDateTime.plusHours(hours).toInstant()));
+ return HoodieActiveTimeline.formatDate(date);
}
}
diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/utils/InputStreamConsumer.java b/hudi-cli/src/main/java/org/apache/hudi/cli/utils/InputStreamConsumer.java
index 73aa45c500d09..a2ebe5769d488 100644
--- a/hudi-cli/src/main/java/org/apache/hudi/cli/utils/InputStreamConsumer.java
+++ b/hudi-cli/src/main/java/org/apache/hudi/cli/utils/InputStreamConsumer.java
@@ -18,18 +18,19 @@
package org.apache.hudi.cli.utils;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
import java.io.BufferedReader;
-import java.io.IOException;
import java.io.InputStream;
import java.io.InputStreamReader;
-import java.util.logging.Logger;
/**
* This class is responsible to read a Process output.
*/
public class InputStreamConsumer extends Thread {
- private static final Logger LOG = Logger.getLogger(InputStreamConsumer.class.getName());
+ private static final Logger LOG = LogManager.getLogger(InputStreamConsumer.class);
private InputStream is;
public InputStreamConsumer(InputStream is) {
@@ -41,13 +42,10 @@ public void run() {
try {
InputStreamReader isr = new InputStreamReader(is);
BufferedReader br = new BufferedReader(isr);
- String line;
- while ((line = br.readLine()) != null) {
- LOG.info(line);
- }
- } catch (IOException ioe) {
- LOG.severe(ioe.toString());
- ioe.printStackTrace();
+ br.lines().forEach(LOG::info);
+ } catch (Exception e) {
+ LOG.fatal(e.toString());
+ e.printStackTrace();
}
}
diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/utils/SparkTempViewProvider.java b/hudi-cli/src/main/java/org/apache/hudi/cli/utils/SparkTempViewProvider.java
index 5e029cd050b34..4f9e4b0d9a9c0 100644
--- a/hudi-cli/src/main/java/org/apache/hudi/cli/utils/SparkTempViewProvider.java
+++ b/hudi-cli/src/main/java/org/apache/hudi/cli/utils/SparkTempViewProvider.java
@@ -19,9 +19,8 @@
package org.apache.hudi.cli.utils;
import org.apache.hudi.exception.HoodieException;
-
-import org.apache.log4j.LogManager;
-import org.apache.log4j.Logger;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.sql.Dataset;
@@ -36,6 +35,7 @@
import java.util.stream.Collectors;
public class SparkTempViewProvider implements TempViewProvider {
+
private static final Logger LOG = LogManager.getLogger(SparkTempViewProvider.class);
private JavaSparkContext jsc;
@@ -46,16 +46,19 @@ public SparkTempViewProvider(String appName) {
SparkConf sparkConf = new SparkConf().setAppName(appName)
.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer").setMaster("local[8]");
jsc = new JavaSparkContext(sparkConf);
- jsc.setLogLevel("ERROR");
-
sqlContext = new SQLContext(jsc);
} catch (Throwable ex) {
// log full stack trace and rethrow. Without this its difficult to debug failures, if any
- LOG.error("unable to initialize spark context ", ex);
+ LOG.warn("unable to initialize spark context ", ex);
throw new HoodieException(ex);
}
}
+ public SparkTempViewProvider(JavaSparkContext jsc, SQLContext sqlContext) {
+ this.jsc = jsc;
+ this.sqlContext = sqlContext;
+ }
+
@Override
public void createOrReplace(String tableName, List headers, List> rows) {
try {
@@ -85,7 +88,7 @@ public void createOrReplace(String tableName, List headers, List
+ sparkLauncher.addJar(new File(libDirectory, library).getAbsolutePath()));
}
return sparkLauncher;
}
- public static JavaSparkContext initJavaSparkConf(String name) {
- return initJavaSparkConf(name, Option.empty(), Option.empty());
- }
-
- public static JavaSparkContext initJavaSparkConf(String name, Option master,
- Option executorMemory) {
- SparkConf sparkConf = new SparkConf().setAppName(name);
+ /**
+ * Get the default spark configuration.
+ *
+ * @param appName - Spark application name
+ * @param sparkMaster - Spark master node name
+ * @return Spark configuration
+ */
+ public static SparkConf getDefaultConf(final String appName, final Option sparkMaster) {
+ final Properties properties = System.getProperties();
+ SparkConf sparkConf = new SparkConf().setAppName(appName);
- String defMaster = master.orElse(sparkConf.getenv(HoodieCliSparkConfig.CLI_SPARK_MASTER));
- if ((null == defMaster) || (defMaster.isEmpty())) {
- sparkConf.setMaster(DEFAULT_SPARK_MASTER);
- } else {
- sparkConf.setMaster(defMaster);
+ // Configure the sparkMaster
+ String sparkMasterNode = DEFAULT_SPARK_MASTER;
+ if (properties.getProperty(HoodieCliSparkConfig.CLI_SPARK_MASTER) != null) {
+ sparkMasterNode = properties.getProperty(HoodieCliSparkConfig.CLI_SPARK_MASTER);
+ }
+ if (sparkMaster.isPresent() && !sparkMaster.get().trim().isEmpty()) {
+ sparkMasterNode = sparkMaster.orElse(sparkMasterNode);
}
+ sparkConf.setMaster(sparkMasterNode);
- sparkConf.set(HoodieCliSparkConfig.CLI_SERIALIZER, "org.apache.spark.serializer.KryoSerializer");
+ // Configure driver
sparkConf.set(HoodieCliSparkConfig.CLI_DRIVER_MAX_RESULT_SIZE, "2g");
sparkConf.set(HoodieCliSparkConfig.CLI_EVENT_LOG_OVERWRITE, "true");
- sparkConf.set(HoodieCliSparkConfig.CLI_EVENT_LOG_ENABLED, "true");
- if (executorMemory.isPresent()) {
- sparkConf.set(HoodieCliSparkConfig.CLI_EXECUTOR_MEMORY, executorMemory.get());
- }
+ sparkConf.set(HoodieCliSparkConfig.CLI_EVENT_LOG_ENABLED, "false");
+ sparkConf.set(HoodieCliSparkConfig.CLI_SERIALIZER, "org.apache.spark.serializer.KryoSerializer");
// Configure hadoop conf
sparkConf.set(HoodieCliSparkConfig.CLI_MAPRED_OUTPUT_COMPRESS, "true");
@@ -91,10 +99,28 @@ public static JavaSparkContext initJavaSparkConf(String name, Option mas
sparkConf.set(HoodieCliSparkConfig.CLI_MAPRED_OUTPUT_COMPRESSION_CODEC, "org.apache.hadoop.io.compress.GzipCodec");
sparkConf.set(HoodieCliSparkConfig.CLI_MAPRED_OUTPUT_COMPRESSION_TYPE, "BLOCK");
+ return sparkConf;
+ }
+
+ public static JavaSparkContext initJavaSparkContext(String name) {
+ return initJavaSparkContext(name, Option.empty(), Option.empty());
+ }
+
+ public static JavaSparkContext initJavaSparkContext(String name, Option master, Option executorMemory) {
+ SparkConf sparkConf = getDefaultConf(name, master);
+ if (executorMemory.isPresent()) {
+ sparkConf.set(HoodieCliSparkConfig.CLI_EXECUTOR_MEMORY, executorMemory.get());
+ }
+
+ return initJavaSparkContext(sparkConf);
+ }
+
+ public static JavaSparkContext initJavaSparkContext(SparkConf sparkConf) {
SparkRDDWriteClient.registerClasses(sparkConf);
JavaSparkContext jsc = new JavaSparkContext(sparkConf);
jsc.hadoopConfiguration().setBoolean(HoodieCliSparkConfig.CLI_PARQUET_ENABLE_SUMMARY_METADATA, false);
FSUtils.prepareHadoopConf(jsc.hadoopConfiguration());
return jsc;
}
+
}
diff --git a/hudi-cli/src/main/resources/META-INF/spring/spring-shell-plugin.xml b/hudi-cli/src/main/resources/META-INF/spring/spring-shell-plugin.xml
deleted file mode 100644
index e28b4f9e887eb..0000000000000
--- a/hudi-cli/src/main/resources/META-INF/spring/spring-shell-plugin.xml
+++ /dev/null
@@ -1,28 +0,0 @@
-
-
-
-
-
-
-
-
diff --git a/hudi-cli/src/main/resources/application.yml b/hudi-cli/src/main/resources/application.yml
new file mode 100644
index 0000000000000..036524c58d5ed
--- /dev/null
+++ b/hudi-cli/src/main/resources/application.yml
@@ -0,0 +1,23 @@
+###
+# 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.
+###
+
+spring:
+ shell:
+ history:
+ enabled: true
+ name: hoodie-cmd.log
\ No newline at end of file
diff --git a/hudi-cli/src/main/resources/banner.txt b/hudi-cli/src/main/resources/banner.txt
new file mode 100644
index 0000000000000..be572b83eb277
--- /dev/null
+++ b/hudi-cli/src/main/resources/banner.txt
@@ -0,0 +1,14 @@
+===================================================================
+* ___ ___ *
+* /\__\ ___ /\ \ ___ *
+* / / / /\__\ / \ \ /\ \ *
+* / /__/ / / / / /\ \ \ \ \ \ *
+* / \ \ ___ / / / / / \ \__\ / \__\ *
+* / /\ \ /\__\ / /__/ ___ / /__/ \ |__| / /\/__/ *
+* \/ \ \/ / / \ \ \ /\__\ \ \ \ / / / /\/ / / *
+* \ / / \ \ / / / \ \ / / / \ /__/ *
+* / / / \ \/ / / \ \/ / / \ \__\ *
+* / / / \ / / \ / / \/__/ *
+* \/__/ \/__/ \/__/ Apache Hudi CLI *
+* *
+===================================================================
\ No newline at end of file
diff --git a/hudi-cli/src/main/resources/log4j2.properties b/hudi-cli/src/main/resources/log4j2.properties
new file mode 100644
index 0000000000000..bc8e5ad56c7d6
--- /dev/null
+++ b/hudi-cli/src/main/resources/log4j2.properties
@@ -0,0 +1,38 @@
+###
+# 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.
+###
+
+status = INFO
+name = HudiCliLog4j2
+
+appender.console.type = Console
+appender.console.name = CONSOLE
+appender.console.layout.type = PatternLayout
+appender.console.layout.pattern = %-4r [%t] %-5p %c %x - %m%n
+
+# Root logger level
+rootLogger.level = warn
+# Root logger referring to console appender
+rootLogger.appenderRef.stdout.ref = CONSOLE
+
+logger.hudi_cli.name = org.apache.hudi.cli
+logger.hudi_cli.level = info
+logger.hudi_common.name = org.apache.hudi.common
+logger.hudi_common.level = info
+
+logger.spark.name = org.apache.spark
+logger.spark.level = info
diff --git a/hudi-cli/src/main/scala/org/apache/hudi/cli/DedupeSparkJob.scala b/hudi-cli/src/main/scala/org/apache/hudi/cli/DedupeSparkJob.scala
index 96944c5c032cd..00e96a3487504 100644
--- a/hudi-cli/src/main/scala/org/apache/hudi/cli/DedupeSparkJob.scala
+++ b/hudi-cli/src/main/scala/org/apache/hudi/cli/DedupeSparkJob.scala
@@ -18,14 +18,14 @@
package org.apache.hudi.cli
import java.util.stream.Collectors
-
import org.apache.hadoop.fs.{FileSystem, FileUtil, Path}
import org.apache.hudi.common.fs.FSUtils
import org.apache.hudi.common.model.{HoodieBaseFile, HoodieRecord}
import org.apache.hudi.common.table.HoodieTableMetaClient
import org.apache.hudi.common.table.view.HoodieTableFileSystemView
import org.apache.hudi.exception.HoodieException
-import org.apache.log4j.Logger
+import org.apache.logging.log4j.LogManager
+import org.apache.logging.log4j.Logger
import org.apache.spark.sql.{DataFrame, Row, SQLContext}
import scala.collection.JavaConversions._
@@ -42,7 +42,7 @@ class DedupeSparkJob(basePath: String,
dedupeType: DeDupeType.Value) {
val sparkHelper = new SparkHelper(sqlContext, fs)
- val LOG = Logger.getLogger(this.getClass)
+ val LOG = LogManager.getLogger(this.getClass)
/**
@@ -55,7 +55,7 @@ class DedupeSparkJob(basePath: String,
s"""
select `${HoodieRecord.RECORD_KEY_METADATA_FIELD}` as dupe_key,
count(*) as dupe_cnt
- from ${tblName}
+ from $tblName
group by `${HoodieRecord.RECORD_KEY_METADATA_FIELD}`
having dupe_cnt > 1
"""
@@ -75,10 +75,10 @@ class DedupeSparkJob(basePath: String,
val tmpTableName = s"htbl_${System.currentTimeMillis()}"
val dedupeTblName = s"${tmpTableName}_dupeKeys"
- val metadata = new HoodieTableMetaClient(fs.getConf, basePath)
+ val metadata = HoodieTableMetaClient.builder().setConf(fs.getConf).setBasePath(basePath).build()
val allFiles = fs.listStatus(new org.apache.hadoop.fs.Path(s"$basePath/$duplicatedPartitionPath"))
- val fsView = new HoodieTableFileSystemView(metadata, metadata.getActiveTimeline.getCommitTimeline.filterCompletedInstants(), allFiles)
+ val fsView = new HoodieTableFileSystemView(metadata, metadata.getActiveTimeline.getCommitsTimeline.filterCompletedInstants(), allFiles)
val latestFiles: java.util.List[HoodieBaseFile] = fsView.getLatestBaseFiles().collect(Collectors.toList[HoodieBaseFile]())
val filteredStatuses = latestFiles.map(f => f.getPath)
LOG.info(s" List of files under partition: ${} => ${filteredStatuses.mkString(" ")}")
@@ -184,10 +184,10 @@ class DedupeSparkJob(basePath: String,
}
def fixDuplicates(dryRun: Boolean = true) = {
- val metadata = new HoodieTableMetaClient(fs.getConf, basePath)
+ val metadata = HoodieTableMetaClient.builder().setConf(fs.getConf).setBasePath(basePath).build()
val allFiles = fs.listStatus(new Path(s"$basePath/$duplicatedPartitionPath"))
- val fsView = new HoodieTableFileSystemView(metadata, metadata.getActiveTimeline.getCommitTimeline.filterCompletedInstants(), allFiles)
+ val fsView = new HoodieTableFileSystemView(metadata, metadata.getActiveTimeline.getCommitsTimeline.filterCompletedInstants(), allFiles)
val latestFiles: java.util.List[HoodieBaseFile] = fsView.getLatestBaseFiles().collect(Collectors.toList[HoodieBaseFile]())
@@ -233,7 +233,7 @@ class DedupeSparkJob(basePath: String,
}
println("No duplicates found & counts are in check!!!! ")
- // 4. Prepare to copy the fixed files back.
+ // 5. Prepare to copy the fixed files back.
fileNameToPathMap.foreach { case (_, filePath) =>
val srcPath = new Path(s"$repairOutputPath/${filePath.getName}")
val dstPath = new Path(s"$basePath/$duplicatedPartitionPath/${filePath.getName}")
diff --git a/hudi-cli/src/main/scala/org/apache/hudi/cli/SparkHelpers.scala b/hudi-cli/src/main/scala/org/apache/hudi/cli/SparkHelpers.scala
index 6859f7038c731..826fbcb4788e7 100644
--- a/hudi-cli/src/main/scala/org/apache/hudi/cli/SparkHelpers.scala
+++ b/hudi-cli/src/main/scala/org/apache/hudi/cli/SparkHelpers.scala
@@ -23,12 +23,11 @@ import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.{FileSystem, Path}
import org.apache.hudi.avro.HoodieAvroWriteSupport
import org.apache.hudi.client.SparkTaskContextSupplier
-import org.apache.hudi.common.HoodieJsonPayload
import org.apache.hudi.common.bloom.{BloomFilter, BloomFilterFactory}
-import org.apache.hudi.common.model.HoodieRecord
-import org.apache.hudi.common.util.ParquetUtils
+import org.apache.hudi.common.model.{HoodieFileFormat, HoodieRecord}
+import org.apache.hudi.common.util.BaseFileUtils
import org.apache.hudi.config.{HoodieIndexConfig, HoodieStorageConfig}
-import org.apache.hudi.io.storage.{HoodieAvroParquetConfig, HoodieParquetWriter}
+import org.apache.hudi.io.storage.{HoodieAvroParquetWriter, HoodieParquetConfig}
import org.apache.parquet.avro.AvroSchemaConverter
import org.apache.parquet.hadoop.metadata.CompressionCodecName
import org.apache.spark.sql.{DataFrame, SQLContext}
@@ -40,17 +39,17 @@ import scala.collection.mutable._
object SparkHelpers {
@throws[Exception]
def skipKeysAndWriteNewFile(instantTime: String, fs: FileSystem, sourceFile: Path, destinationFile: Path, keysToSkip: Set[String]) {
- val sourceRecords = ParquetUtils.readAvroRecords(fs.getConf, sourceFile)
+ val sourceRecords = BaseFileUtils.getInstance(HoodieFileFormat.PARQUET).readAvroRecords(fs.getConf, sourceFile)
val schema: Schema = sourceRecords.get(0).getSchema
- val filter: BloomFilter = BloomFilterFactory.createBloomFilter(HoodieIndexConfig.DEFAULT_BLOOM_FILTER_NUM_ENTRIES.toInt, HoodieIndexConfig.DEFAULT_BLOOM_FILTER_FPP.toDouble,
- HoodieIndexConfig.DEFAULT_HOODIE_BLOOM_INDEX_FILTER_DYNAMIC_MAX_ENTRIES.toInt, HoodieIndexConfig.DEFAULT_BLOOM_INDEX_FILTER_TYPE);
- val writeSupport: HoodieAvroWriteSupport = new HoodieAvroWriteSupport(new AvroSchemaConverter().convert(schema), schema, filter)
- val parquetConfig: HoodieAvroParquetConfig = new HoodieAvroParquetConfig(writeSupport, CompressionCodecName.GZIP, HoodieStorageConfig.DEFAULT_PARQUET_BLOCK_SIZE_BYTES.toInt, HoodieStorageConfig.DEFAULT_PARQUET_PAGE_SIZE_BYTES.toInt, HoodieStorageConfig.DEFAULT_PARQUET_FILE_MAX_BYTES.toInt, fs.getConf, HoodieStorageConfig.DEFAULT_STREAM_COMPRESSION_RATIO.toDouble)
+ val filter: BloomFilter = BloomFilterFactory.createBloomFilter(HoodieIndexConfig.BLOOM_FILTER_NUM_ENTRIES_VALUE.defaultValue.toInt, HoodieIndexConfig.BLOOM_FILTER_FPP_VALUE.defaultValue.toDouble,
+ HoodieIndexConfig.BLOOM_INDEX_FILTER_DYNAMIC_MAX_ENTRIES.defaultValue.toInt, HoodieIndexConfig.BLOOM_FILTER_TYPE.defaultValue);
+ val writeSupport: HoodieAvroWriteSupport = new HoodieAvroWriteSupport(new AvroSchemaConverter(fs.getConf).convert(schema), schema, org.apache.hudi.common.util.Option.of(filter))
+ val parquetConfig: HoodieParquetConfig[HoodieAvroWriteSupport] = new HoodieParquetConfig(writeSupport, CompressionCodecName.GZIP, HoodieStorageConfig.PARQUET_BLOCK_SIZE.defaultValue.toInt, HoodieStorageConfig.PARQUET_PAGE_SIZE.defaultValue.toInt, HoodieStorageConfig.PARQUET_MAX_FILE_SIZE.defaultValue.toInt, fs.getConf, HoodieStorageConfig.PARQUET_COMPRESSION_RATIO_FRACTION.defaultValue.toDouble)
// Add current classLoad for config, if not will throw classNotFound of 'HoodieWrapperFileSystem'.
parquetConfig.getHadoopConf().setClassLoader(Thread.currentThread.getContextClassLoader)
- val writer = new HoodieParquetWriter[HoodieJsonPayload, IndexedRecord](instantTime, destinationFile, parquetConfig, schema, new SparkTaskContextSupplier())
+ val writer = new HoodieAvroParquetWriter(destinationFile, parquetConfig, instantTime, new SparkTaskContextSupplier(), true)
for (rec <- sourceRecords) {
val key: String = rec.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString
if (!keysToSkip.contains(key)) {
@@ -125,7 +124,7 @@ class SparkHelper(sqlContext: SQLContext, fs: FileSystem) {
* @return
*/
def fileKeysAgainstBF(conf: Configuration, sqlContext: SQLContext, file: String): Boolean = {
- val bf = ParquetUtils.readBloomFilterFromParquetMetadata(conf, new Path(file))
+ val bf = BaseFileUtils.getInstance(HoodieFileFormat.PARQUET).readBloomFilterFromMetadata(conf, new Path(file))
val foundCount = sqlContext.parquetFile(file)
.select(s"`${HoodieRecord.RECORD_KEY_METADATA_FIELD}`")
.collect().count(r => !bf.mightContain(r.getString(0)))
diff --git a/hudi-cli/src/test/java/org/apache/hudi/cli/TestSparkUtil.java b/hudi-cli/src/test/java/org/apache/hudi/cli/TestSparkUtil.java
new file mode 100644
index 0000000000000..4dcd15156baf1
--- /dev/null
+++ b/hudi-cli/src/test/java/org/apache/hudi/cli/TestSparkUtil.java
@@ -0,0 +1,46 @@
+/*
+ * 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.cli;
+
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.cli.utils.SparkUtil;
+import org.apache.spark.SparkConf;
+
+import org.apache.spark.launcher.SparkLauncher;
+import org.junit.jupiter.api.Test;
+
+import java.net.URISyntaxException;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+
+public class TestSparkUtil {
+
+ @Test
+ public void testInitSparkLauncher() throws URISyntaxException {
+ SparkLauncher sparkLauncher = SparkUtil.initLauncher(null);
+ assertNotNull(sparkLauncher);
+ }
+
+ @Test
+ public void testGetDefaultSparkConf() {
+ SparkConf sparkConf = SparkUtil.getDefaultConf("test-spark-app", Option.of(""));
+ assertEquals(SparkUtil.DEFAULT_SPARK_MASTER, sparkConf.get("spark.master"));
+ }
+}
diff --git a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestArchivedCommitsCommand.java b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestArchivedCommitsCommand.java
index 5c27636da18c8..b642c1b3f8ebb 100644
--- a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestArchivedCommitsCommand.java
+++ b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestArchivedCommitsCommand.java
@@ -21,25 +21,29 @@
import org.apache.hudi.cli.HoodieCLI;
import org.apache.hudi.cli.HoodiePrintHelper;
import org.apache.hudi.cli.TableHeader;
-import org.apache.hudi.cli.testutils.AbstractShellIntegrationTest;
+import org.apache.hudi.cli.functional.CLIFunctionalTestHarness;
import org.apache.hudi.cli.testutils.HoodieTestCommitMetadataGenerator;
import org.apache.hudi.cli.testutils.HoodieTestCommitUtilities;
+import org.apache.hudi.cli.testutils.ShellEvaluationResultUtil;
+import org.apache.hudi.client.HoodieTimelineArchiver;
import org.apache.hudi.common.model.HoodieCommitMetadata;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
-import org.apache.hudi.config.HoodieCompactionConfig;
+import org.apache.hudi.common.table.view.FileSystemViewStorageConfig;
+import org.apache.hudi.common.testutils.HoodieTestUtils;
+import org.apache.hudi.config.HoodieArchivalConfig;
+import org.apache.hudi.config.HoodieCleanConfig;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.table.HoodieSparkTable;
-import org.apache.hudi.table.HoodieTimelineArchiveLog;
-import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Tag;
import org.junit.jupiter.api.Test;
-import org.springframework.shell.core.CommandResult;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.boot.test.context.SpringBootTest;
+import org.springframework.shell.Shell;
-import java.io.File;
-import java.io.IOException;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
@@ -50,30 +54,36 @@
/**
* Test Cases for {@link ArchivedCommitsCommand}.
*/
-public class TestArchivedCommitsCommand extends AbstractShellIntegrationTest {
+@Tag("functional")
+@SpringBootTest(properties = {"spring.shell.interactive.enabled=false", "spring.shell.command.script.enabled=false"})
+public class TestArchivedCommitsCommand extends CLIFunctionalTestHarness {
+
+ @Autowired
+ private Shell shell;
private String tablePath;
@BeforeEach
public void init() throws Exception {
- initDFS();
- jsc.hadoopConfiguration().addResource(dfs.getConf());
- HoodieCLI.conf = dfs.getConf();
+ HoodieCLI.conf = hadoopConf();
// Create table and connect
- String tableName = "test_table";
- tablePath = basePath + File.separator + tableName;
+ String tableName = tableName();
+ tablePath = tablePath(tableName);
new TableCommand().createTable(
tablePath, tableName,
"COPY_ON_WRITE", "", 1, "org.apache.hudi.common.model.HoodieAvroPayload");
- metaClient = HoodieCLI.getTableMetaClient();
+ HoodieTableMetaClient metaClient = HoodieCLI.getTableMetaClient();
// Generate archive
HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(tablePath)
.withSchema(HoodieTestCommitMetadataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
- .withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(2, 3).build())
+ .withArchivalConfig(HoodieArchivalConfig.newBuilder().archiveCommitsWith(2, 3).build())
+ .withCleanConfig(HoodieCleanConfig.newBuilder().retainCommits(1).build())
+ .withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder()
+ .withRemoteServerPort(timelineServicePort).build())
.forTable("test-trip-table").build();
// Create six commits
@@ -81,26 +91,26 @@ public void init() throws Exception {
String timestamp = String.valueOf(i);
// Requested Compaction
HoodieTestCommitMetadataGenerator.createCompactionAuxiliaryMetadata(tablePath,
- new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, timestamp), dfs.getConf());
+ new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, timestamp), hadoopConf());
// Inflight Compaction
HoodieTestCommitMetadataGenerator.createCompactionAuxiliaryMetadata(tablePath,
- new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, timestamp), dfs.getConf());
- HoodieTestCommitMetadataGenerator.createCommitFileWithMetadata(tablePath, timestamp, dfs.getConf());
+ new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, timestamp), hadoopConf());
+ HoodieTestCommitMetadataGenerator.createCommitFileWithMetadata(tablePath, timestamp, hadoopConf());
}
+ // Simulate a compaction commit in metadata table timeline
+ // so the archival in data table can happen
+ HoodieTestUtils.createCompactionCommitInMetadataTable(
+ hadoopConf(), metaClient.getFs(), tablePath, "105");
+
metaClient = HoodieTableMetaClient.reload(metaClient);
// reload the timeline and get all the commits before archive
metaClient.getActiveTimeline().reload().getAllCommitsTimeline().filterCompletedInstants();
// archive
- HoodieSparkTable table = HoodieSparkTable.create(cfg, context, metaClient);
- HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, table);
- archiveLog.archiveIfRequired(context);
- }
-
- @AfterEach
- public void clean() throws IOException {
- cleanupDFS();
+ HoodieSparkTable table = HoodieSparkTable.create(cfg, context(), metaClient);
+ HoodieTimelineArchiver archiver = new HoodieTimelineArchiver(cfg, table);
+ archiver.archiveIfRequired(context());
}
/**
@@ -108,8 +118,8 @@ public void clean() throws IOException {
*/
@Test
public void testShowArchivedCommits() {
- CommandResult cr = getShell().executeCommand("show archived commit stats");
- assertTrue(cr.isSuccess());
+ Object result = shell.evaluate(() -> "show archived commit stats");
+ assertTrue(ShellEvaluationResultUtil.isSuccess(result));
TableHeader header = new TableHeader().addTableHeaderField("action").addTableHeaderField("instant")
.addTableHeaderField("partition").addTableHeaderField("file_id").addTableHeaderField("prev_instant")
@@ -150,7 +160,7 @@ public void testShowArchivedCommits() {
String expectedResult = HoodiePrintHelper.print(
header, new HashMap<>(), "", false, -1, false, rows);
expectedResult = removeNonWordAndStripSpace(expectedResult);
- String got = removeNonWordAndStripSpace(cr.getResult().toString());
+ String got = removeNonWordAndStripSpace(result.toString());
assertEquals(expectedResult, got);
}
@@ -159,8 +169,8 @@ public void testShowArchivedCommits() {
*/
@Test
public void testShowCommits() throws Exception {
- CommandResult cr = getShell().executeCommand("show archived commits");
- assertTrue(cr.isSuccess());
+ Object cmdResult = shell.evaluate(() -> "show archived commits");
+ assertTrue(ShellEvaluationResultUtil.isSuccess(cmdResult));
final List rows = new ArrayList<>();
// Test default skipMetadata and limit 10
@@ -175,12 +185,12 @@ public void testShowCommits() throws Exception {
rows.add(new Comparable[] {"103", "commit"});
String expected = HoodiePrintHelper.print(header, new HashMap<>(), "", false, 10, false, rows);
expected = removeNonWordAndStripSpace(expected);
- String got = removeNonWordAndStripSpace(cr.getResult().toString());
+ String got = removeNonWordAndStripSpace(cmdResult.toString());
assertEquals(expected, got);
// Test with Metadata and no limit
- cr = getShell().executeCommand("show archived commits --skipMetadata false --limit -1");
- assertTrue(cr.isSuccess());
+ cmdResult = shell.evaluate(() -> "show archived commits --skipMetadata false --limit 0");
+ assertTrue(ShellEvaluationResultUtil.isSuccess(cmdResult));
rows.clear();
@@ -195,9 +205,9 @@ public void testShowCommits() throws Exception {
rows.add(result);
}
header = header.addTableHeaderField("CommitDetails");
- expected = HoodiePrintHelper.print(header, new HashMap<>(), "", false, -1, false, rows);
+ expected = HoodiePrintHelper.print(header, new HashMap<>(), "", false, 0, false, rows);
expected = removeNonWordAndStripSpace(expected);
- got = removeNonWordAndStripSpace(cr.getResult().toString());
+ got = removeNonWordAndStripSpace(cmdResult.toString());
assertEquals(expected, got);
}
}
diff --git a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestCleansCommand.java b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestCleansCommand.java
index 2311aaa22f3fa..f0ed1787e21f8 100644
--- a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestCleansCommand.java
+++ b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestCleansCommand.java
@@ -23,9 +23,12 @@
import org.apache.hudi.cli.HoodiePrintHelper;
import org.apache.hudi.cli.HoodieTableHeaderFields;
import org.apache.hudi.cli.TableHeader;
-import org.apache.hudi.cli.testutils.AbstractShellIntegrationTest;
+import org.apache.hudi.cli.functional.CLIFunctionalTestHarness;
import org.apache.hudi.cli.testutils.HoodieTestCommitMetadataGenerator;
+import org.apache.hudi.cli.testutils.ShellEvaluationResultUtil;
+import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.HoodieCleaningPolicy;
+import org.apache.hudi.common.model.HoodieCommitMetadata;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
@@ -34,19 +37,25 @@
import org.apache.hudi.common.table.timeline.TimelineMetadataUtils;
import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion;
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
+import org.apache.hudi.common.testutils.HoodieTestTable;
import org.apache.hudi.common.util.Option;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Tag;
import org.junit.jupiter.api.Test;
-import org.springframework.shell.core.CommandResult;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.boot.test.context.SpringBootTest;
+import org.springframework.shell.Shell;
-import java.io.File;
import java.io.IOException;
import java.net.URL;
import java.util.ArrayList;
+import java.util.Collections;
import java.util.HashMap;
import java.util.List;
+import java.util.Map;
import java.util.UUID;
import static org.junit.jupiter.api.Assertions.assertEquals;
@@ -56,17 +65,22 @@
/**
* Test Cases for {@link CleansCommand}.
*/
-public class TestCleansCommand extends AbstractShellIntegrationTest {
+@Tag("functional")
+@SpringBootTest(properties = {"spring.shell.interactive.enabled=false", "spring.shell.command.script.enabled=false"})
+public class TestCleansCommand extends CLIFunctionalTestHarness {
+
+ @Autowired
+ private Shell shell;
- private String tablePath;
private URL propsFilePath;
+ private HoodieTableMetaClient metaClient;
@BeforeEach
public void init() throws Exception {
- HoodieCLI.conf = jsc.hadoopConfiguration();
+ HoodieCLI.conf = hadoopConf();
- String tableName = "test_table";
- tablePath = basePath + File.separator + tableName;
+ String tableName = tableName();
+ String tablePath = tablePath(tableName);
propsFilePath = TestCleansCommand.class.getClassLoader().getResource("clean.properties");
// Create table and connect
@@ -79,7 +93,8 @@ public void init() throws Exception {
metaClient = HoodieCLI.getTableMetaClient();
String fileId1 = UUID.randomUUID().toString();
String fileId2 = UUID.randomUUID().toString();
- HoodieTestDataGenerator.writePartitionMetadata(fs, HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS, tablePath);
+ FileSystem fs = FSUtils.getFs(basePath(), hadoopConf());
+ HoodieTestDataGenerator.writePartitionMetadataDeprecated(fs, HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS, tablePath);
// Create four commits
for (int i = 100; i < 104; i++) {
@@ -90,8 +105,11 @@ public void init() throws Exception {
// Inflight Compaction
HoodieTestCommitMetadataGenerator.createCompactionAuxiliaryMetadata(tablePath,
new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, timestamp), conf);
+
+ Map extraCommitMetadata =
+ Collections.singletonMap(HoodieCommitMetadata.SCHEMA_KEY, HoodieTestTable.PHONY_TABLE_SCHEMA);
HoodieTestCommitMetadataGenerator.createCommitFileWithMetadata(tablePath, timestamp, conf, fileId1, fileId2,
- Option.empty(), Option.empty());
+ Option.empty(), Option.empty(), extraCommitMetadata);
}
metaClient = HoodieTableMetaClient.reload(metaClient);
@@ -108,12 +126,12 @@ public void testShowCleans() throws Exception {
assertNotNull(propsFilePath, "Not found properties file");
// First, run clean
- SparkMain.clean(jsc, HoodieCLI.basePath, propsFilePath.getPath(), new ArrayList<>());
+ SparkMain.clean(jsc(), HoodieCLI.basePath, propsFilePath.getPath(), new ArrayList<>());
assertEquals(1, metaClient.getActiveTimeline().reload().getCleanerTimeline().getInstants().count(),
"Loaded 1 clean and the count should match");
- CommandResult cr = getShell().executeCommand("cleans show");
- assertTrue(cr.isSuccess());
+ Object result = shell.evaluate(() -> "cleans show");
+ assertTrue(ShellEvaluationResultUtil.isSuccess(result));
HoodieInstant clean = metaClient.getActiveTimeline().reload().getCleanerTimeline().getInstants().findFirst().orElse(null);
assertNotNull(clean);
@@ -131,7 +149,7 @@ public void testShowCleans() throws Exception {
String expected = HoodiePrintHelper.print(header, new HashMap<>(), "", false, -1, false, rows);
expected = removeNonWordAndStripSpace(expected);
- String got = removeNonWordAndStripSpace(cr.getResult().toString());
+ String got = removeNonWordAndStripSpace(result.toString());
assertEquals(expected, got);
}
@@ -139,19 +157,19 @@ public void testShowCleans() throws Exception {
* Test case for show partitions of a clean instant.
*/
@Test
- public void testShowCleanPartitions() throws IOException {
+ public void testShowCleanPartitions() {
// Check properties file exists.
assertNotNull(propsFilePath, "Not found properties file");
// First, run clean with two partition
- SparkMain.clean(jsc, HoodieCLI.basePath, propsFilePath.toString(), new ArrayList<>());
+ SparkMain.clean(jsc(), HoodieCLI.basePath, propsFilePath.toString(), new ArrayList<>());
assertEquals(1, metaClient.getActiveTimeline().reload().getCleanerTimeline().getInstants().count(),
"Loaded 1 clean and the count should match");
HoodieInstant clean = metaClient.getActiveTimeline().reload().getCleanerTimeline().getInstants().findFirst().get();
- CommandResult cr = getShell().executeCommand("clean showpartitions --clean " + clean.getTimestamp());
- assertTrue(cr.isSuccess());
+ Object result = shell.evaluate(() -> "clean showpartitions --clean " + clean.getTimestamp());
+ assertTrue(ShellEvaluationResultUtil.isSuccess(result));
TableHeader header = new TableHeader().addTableHeaderField(HoodieTableHeaderFields.HEADER_PARTITION_PATH)
.addTableHeaderField(HoodieTableHeaderFields.HEADER_CLEANING_POLICY)
@@ -169,7 +187,7 @@ public void testShowCleanPartitions() throws IOException {
String expected = HoodiePrintHelper.print(header, new HashMap<>(), "", false, -1, false, rows);
expected = removeNonWordAndStripSpace(expected);
- String got = removeNonWordAndStripSpace(cr.getResult().toString());
+ String got = removeNonWordAndStripSpace(result.toString());
assertEquals(expected, got);
}
diff --git a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestCommitsCommand.java b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestCommitsCommand.java
index 84b357622948e..7e504488a2dfd 100644
--- a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestCommitsCommand.java
+++ b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestCommitsCommand.java
@@ -22,28 +22,39 @@
import org.apache.hudi.cli.HoodiePrintHelper;
import org.apache.hudi.cli.HoodieTableHeaderFields;
import org.apache.hudi.cli.TableHeader;
-import org.apache.hudi.cli.testutils.AbstractShellIntegrationTest;
+import org.apache.hudi.cli.functional.CLIFunctionalTestHarness;
import org.apache.hudi.cli.testutils.HoodieTestCommitMetadataGenerator;
+import org.apache.hudi.cli.testutils.HoodieTestReplaceCommitMetadataGenerator;
+import org.apache.hudi.cli.testutils.ShellEvaluationResultUtil;
+import org.apache.hudi.client.HoodieTimelineArchiver;
+import org.apache.hudi.common.config.HoodieMetadataConfig;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion;
+import org.apache.hudi.common.table.view.FileSystemViewStorageConfig;
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
import org.apache.hudi.common.testutils.HoodieTestUtils;
import org.apache.hudi.common.util.NumericUtils;
import org.apache.hudi.common.util.Option;
-import org.apache.hudi.config.HoodieCompactionConfig;
+import org.apache.hudi.config.HoodieArchivalConfig;
+import org.apache.hudi.config.HoodieCleanConfig;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.table.HoodieSparkTable;
-import org.apache.hudi.table.HoodieTimelineArchiveLog;
import org.apache.hadoop.fs.FileSystem;
import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Tag;
import org.junit.jupiter.api.Test;
-import org.springframework.shell.core.CommandResult;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.EnumSource;
+import org.junit.jupiter.params.provider.ValueSource;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.boot.test.context.SpringBootTest;
+import org.springframework.shell.Shell;
-import java.io.File;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
@@ -54,27 +65,38 @@
import java.util.function.Function;
import java.util.stream.Collectors;
+import static org.apache.hudi.common.testutils.HoodieTestUtils.createCompactionCommitInMetadataTable;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertTrue;
/**
* Test class for {@link org.apache.hudi.cli.commands.CommitsCommand}.
*/
-public class TestCommitsCommand extends AbstractShellIntegrationTest {
+@Tag("functional")
+@SpringBootTest(properties = {"spring.shell.interactive.enabled=false", "spring.shell.command.script.enabled=false"})
+public class TestCommitsCommand extends CLIFunctionalTestHarness {
- private String tableName;
- private String tablePath;
+ @Autowired
+ private Shell shell;
+
+ private String tableName1;
+ private String tableName2;
+ private String tablePath1;
+ private String tablePath2;
+ private HoodieTableMetaClient metaClient;
@BeforeEach
public void init() throws IOException {
- tableName = "test_table";
- tablePath = basePath + File.separator + tableName;
-
- HoodieCLI.conf = jsc.hadoopConfiguration();
+ tableName1 = tableName("_1");
+ tableName2 = tableName("_2");
+ tablePath1 = tablePath(tableName1);
+ tablePath2 = tablePath(tableName2);
+ HoodieCLI.conf = hadoopConf();
// Create table and connect
new TableCommand().createTable(
- tablePath, tableName, HoodieTableType.COPY_ON_WRITE.name(),
+ tablePath1, tableName1, HoodieTableType.COPY_ON_WRITE.name(),
"", TimelineLayoutVersion.VERSION_1, "org.apache.hudi.common.model.HoodieAvroPayload");
+ metaClient = HoodieCLI.getTableMetaClient();
}
private LinkedHashMap generateData() throws Exception {
@@ -87,7 +109,7 @@ private LinkedHashMap generateData() throws Exception {
for (Map.Entry entry : data.entrySet()) {
String key = entry.getKey();
Integer[] value = entry.getValue();
- HoodieTestCommitMetadataGenerator.createCommitFileWithMetadata(tablePath, key, jsc.hadoopConfiguration(),
+ HoodieTestCommitMetadataGenerator.createCommitFileWithMetadata(tablePath1, key, hadoopConf(),
Option.of(value[0]), Option.of(value[1]));
}
@@ -97,10 +119,46 @@ private LinkedHashMap generateData() throws Exception {
return data;
}
+ /*
+ * generates both replace commit and commit data
+ * */
+ private LinkedHashMap generateMixedData() throws Exception {
+ // generate data and metadata
+ LinkedHashMap replaceCommitData = new LinkedHashMap<>();
+ replaceCommitData.put(new HoodieInstant(false, HoodieTimeline.REPLACE_COMMIT_ACTION, "103"), new Integer[] {15, 10});
+
+ LinkedHashMap commitData = new LinkedHashMap<>();
+ commitData.put(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "102"), new Integer[] {15, 10});
+ commitData.put(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "101"), new Integer[] {20, 10});
+
+ for (Map.Entry entry : commitData.entrySet()) {
+ String key = entry.getKey().getTimestamp();
+ Integer[] value = entry.getValue();
+ HoodieTestCommitMetadataGenerator.createCommitFileWithMetadata(tablePath1, key, hadoopConf(),
+ Option.of(value[0]), Option.of(value[1]));
+ }
+
+ for (Map.Entry entry : replaceCommitData.entrySet()) {
+ String key = entry.getKey().getTimestamp();
+ Integer[] value = entry.getValue();
+ HoodieTestReplaceCommitMetadataGenerator.createReplaceCommitFileWithMetadata(tablePath1, key,
+ Option.of(value[0]), Option.of(value[1]), metaClient);
+ }
+
+ metaClient = HoodieTableMetaClient.reload(HoodieCLI.getTableMetaClient());
+ assertEquals(3, metaClient.reloadActiveTimeline().getCommitsTimeline().countInstants(),
+ "There should be 3 commits");
+
+ LinkedHashMap data = replaceCommitData;
+ data.putAll(commitData);
+
+ return data;
+ }
+
private String generateExpectData(int records, Map data) throws IOException {
- FileSystem fs = FileSystem.get(jsc.hadoopConfiguration());
+ FileSystem fs = FileSystem.get(hadoopConf());
List partitionPaths =
- FSUtils.getAllPartitionFoldersThreeLevelsDown(fs, tablePath);
+ FSUtils.getAllPartitionFoldersThreeLevelsDown(fs, tablePath1);
int partitions = partitionPaths.size();
// default pre-commit is not null, file add always be 0 and update always be partition nums
@@ -113,27 +171,15 @@ private String generateExpectData(int records, Map data) thro
data.forEach((key, value) -> {
for (int i = 0; i < records; i++) {
// there are more than 1 partitions, so need to * partitions
- rows.add(new Comparable[]{key, partitions * HoodieTestCommitMetadataGenerator.DEFAULT_TOTAL_WRITE_BYTES,
+ rows.add(new Comparable[] {key, partitions * HoodieTestCommitMetadataGenerator.DEFAULT_TOTAL_WRITE_BYTES,
fileAdded, fileUpdated, partitions, partitions * value[0], partitions * value[1], errors});
}
});
final Map> fieldNameToConverterMap = new HashMap<>();
- fieldNameToConverterMap.put(HoodieTableHeaderFields.HEADER_TOTAL_BYTES_WRITTEN, entry -> {
- return NumericUtils.humanReadableByteCount((Double.valueOf(entry.toString())));
- });
-
- final TableHeader header = new TableHeader()
- .addTableHeaderField(HoodieTableHeaderFields.HEADER_COMMIT_TIME)
- .addTableHeaderField(HoodieTableHeaderFields.HEADER_TOTAL_BYTES_WRITTEN)
- .addTableHeaderField(HoodieTableHeaderFields.HEADER_TOTAL_FILES_ADDED)
- .addTableHeaderField(HoodieTableHeaderFields.HEADER_TOTAL_FILES_UPDATED)
- .addTableHeaderField(HoodieTableHeaderFields.HEADER_TOTAL_PARTITIONS_WRITTEN)
- .addTableHeaderField(HoodieTableHeaderFields.HEADER_TOTAL_RECORDS_WRITTEN)
- .addTableHeaderField(HoodieTableHeaderFields.HEADER_TOTAL_UPDATE_RECORDS_WRITTEN)
- .addTableHeaderField(HoodieTableHeaderFields.HEADER_TOTAL_ERRORS);
+ fieldNameToConverterMap.put(HoodieTableHeaderFields.HEADER_TOTAL_BYTES_WRITTEN, entry -> NumericUtils.humanReadableByteCount((Double.parseDouble(entry.toString()))));
- return HoodiePrintHelper.print(header, fieldNameToConverterMap, "", false,
+ return HoodiePrintHelper.print(HoodieTableHeaderFields.getTableHeader(), fieldNameToConverterMap, "", false,
-1, false, rows);
}
@@ -144,24 +190,84 @@ private String generateExpectData(int records, Map