diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/CompactionCommand.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/CompactionCommand.java index b5f344e13..866e9315b 100644 --- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/CompactionCommand.java +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/CompactionCommand.java @@ -45,6 +45,7 @@ import org.apache.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.CliAvailabilityIndicator; import org.springframework.shell.core.annotation.CliCommand; import org.springframework.shell.core.annotation.CliOption; import org.springframework.stereotype.Component; @@ -54,6 +55,12 @@ public class CompactionCommand implements CommandMarker { private static Logger log = LogManager.getLogger(HDFSParquetImportCommand.class); + @CliAvailabilityIndicator({"compactions show all", "compaction show", "compaction run", "compaction schedule"}) + public boolean isAvailable() { + return (HoodieCLI.tableMetadata != null) + && (HoodieCLI.tableMetadata.getTableType() == HoodieTableType.MERGE_ON_READ); + } + @CliCommand(value = "compactions show all", help = "Shows all compactions that are in active timeline") public String compactionsAll( @CliOption(key = { diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/DatasetsCommand.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/DatasetsCommand.java index 1bcdfa5bc..e370f5b42 100644 --- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/DatasetsCommand.java +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/DatasetsCommand.java @@ -17,9 +17,17 @@ package com.uber.hoodie.cli.commands; import com.uber.hoodie.cli.HoodieCLI; +import com.uber.hoodie.cli.HoodiePrintHelper; +import com.uber.hoodie.cli.TableHeader; +import com.uber.hoodie.common.model.HoodieTableType; import com.uber.hoodie.common.table.HoodieTableMetaClient; +import com.uber.hoodie.exception.DatasetNotFoundException; import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; import org.springframework.shell.core.CommandMarker; +import org.springframework.shell.core.annotation.CliAvailabilityIndicator; import org.springframework.shell.core.annotation.CliCommand; import org.springframework.shell.core.annotation.CliOption; import org.springframework.stereotype.Component; @@ -37,4 +45,67 @@ public class DatasetsCommand implements CommandMarker { HoodieCLI.state = HoodieCLI.CLIState.DATASET; return "Metadata for table " + HoodieCLI.tableMetadata.getTableConfig().getTableName() + " loaded"; } + + /** + * Create a Hoodie Table if it does not exist + * + * @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") + public String createTable( + @CliOption(key = {"path"}, mandatory = true, help = "Base Path of the dataset") final String path, + @CliOption(key = {"tableName"}, mandatory = true, help = "Hoodie Table Name") final String name, + @CliOption(key = {"tableType"}, unspecifiedDefaultValue = "COPY_ON_WRITE", + help = "Hoodie Table Type. Must be one of : COPY_ON_WRITE or MERGE_ON_READ") final String tableTypeStr, + @CliOption(key = {"payloadClass"}, unspecifiedDefaultValue = "com.uber.hoodie.common.model.HoodieAvroPayload", + help = "Payload Class") final String payloadClass) throws IOException { + + boolean initialized = HoodieCLI.initConf(); + HoodieCLI.initFS(initialized); + + boolean existing = false; + try { + new HoodieTableMetaClient(HoodieCLI.conf, path); + existing = true; + } catch (DatasetNotFoundException dfe) { + // expected + } + + // Do not touch table that already exist + if (existing) { + throw new IllegalStateException("Dataset already existing in path : " + path); + } + + final HoodieTableType tableType = HoodieTableType.valueOf(tableTypeStr); + HoodieTableMetaClient.initTableType(HoodieCLI.conf, path, tableType, name, payloadClass); + + // Now connect to ensure loading works + return connect(path); + } + + @CliAvailabilityIndicator({"desc"}) + public boolean isDescAvailable() { + return HoodieCLI.tableMetadata != null; + } + + /** + * Describes table properties + */ + @CliCommand(value = "desc", help = "Describle Hoodie Table properties") + public String descTable() { + TableHeader header = new TableHeader() + .addTableHeaderField("Property") + .addTableHeaderField("Value"); + List rows = new ArrayList<>(); + rows.add(new Comparable[]{"basePath", HoodieCLI.tableMetadata.getBasePath()}); + rows.add(new Comparable[]{"metaPath", HoodieCLI.tableMetadata.getMetaPath()}); + rows.add(new Comparable[]{"fileSystem", HoodieCLI.tableMetadata.getFs().getScheme()}); + HoodieCLI.tableMetadata.getTableConfig().getProps().entrySet().forEach(e -> { + rows.add(new Comparable[]{e.getKey(), e.getValue()}); + }); + return HoodiePrintHelper.print(header, new HashMap<>(), "", false, -1, false, rows); + } } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTableConfig.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTableConfig.java index 56921fb0c..007a0f825 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTableConfig.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTableConfig.java @@ -23,7 +23,9 @@ import com.uber.hoodie.exception.HoodieIOException; import java.io.IOException; import java.io.Serializable; import java.util.Date; +import java.util.Map; import java.util.Properties; +import java.util.stream.Collectors; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; @@ -172,4 +174,9 @@ public class HoodieTableConfig implements Serializable { public String getArchivelogFolder() { return props.getProperty(HOODIE_ARCHIVELOG_FOLDER_PROP_NAME, DEFAULT_ARCHIVELOG_FOLDER); } + + public Map getProps() { + return props.entrySet().stream().collect( + Collectors.toMap(e -> String.valueOf(e.getKey()), e -> String.valueOf(e.getValue()))); + } }