1
0

[HUDI-508] Standardizing on "Table" instead of "Dataset" across code (#1197)

- Docs were talking about storage types before, cWiki moved to "Table"
 - Most of code already has HoodieTable, HoodieTableMetaClient - correct naming
 - Replacing renaming use of dataset across code/comments
 - Few usages in comments and use of Spark SQL DataSet remain unscathed
This commit is contained in:
vinoth chandar
2020-01-07 12:52:32 -08:00
committed by GitHub
parent 8306f749a2
commit 9706f659db
73 changed files with 298 additions and 298 deletions

View File

@@ -48,7 +48,7 @@ public class HoodieCLI {
* Enum for CLI state.
*/
public enum CLIState {
INIT, DATASET, SYNC
INIT, TABLE, SYNC
}
public static void setConsistencyGuardConfig(ConsistencyGuardConfig config) {
@@ -100,7 +100,7 @@ public class HoodieCLI {
*/
public static HoodieTableMetaClient getTableMetaClient() {
if (tableMetadata == null) {
throw new NullPointerException("There is no hudi dataset. Please use connect command to set dataset first");
throw new NullPointerException("There is no hudi table. Please use connect command to set table first");
}
return tableMetadata;
}

View File

@@ -37,7 +37,7 @@ public class HoodiePrompt extends DefaultPromptProvider {
switch (HoodieCLI.state) {
case INIT:
return "hudi->";
case DATASET:
case TABLE:
return "hudi:" + tableName + "->";
case SYNC:
return "hudi:" + tableName + " <==> " + HoodieCLI.syncTableMetadata.getTableConfig().getTableName() + "->";

View File

@@ -206,8 +206,8 @@ public class CommitsCommand implements CommandMarker {
return HoodiePrintHelper.print(header, new HashMap<>(), sortByField, descending, limit, headerOnly, rows);
}
@CliCommand(value = "commits compare", help = "Compare commits with another Hoodie dataset")
public String compareCommits(@CliOption(key = {"path"}, help = "Path of the dataset to compare to") final String path)
@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)
throws Exception {
HoodieTableMetaClient source = HoodieCLI.getTableMetaClient();
@@ -234,8 +234,8 @@ public class CommitsCommand implements CommandMarker {
}
}
@CliCommand(value = "commits sync", help = "Compare commits with another Hoodie dataset")
public String syncCommits(@CliOption(key = {"path"}, help = "Path of the dataset to compare to") final String path) {
@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);
HoodieCLI.state = HoodieCLI.CLIState.SYNC;
return "Load sync state between " + HoodieCLI.getTableMetaClient().getTableConfig().getTableName() + " and "

View File

@@ -145,7 +145,7 @@ public class CompactionCommand implements CommandMarker {
@CliCommand(value = "compaction show", help = "Shows compaction details for a specific compaction instant")
public String compactionShow(
@CliOption(key = "instant", mandatory = true,
help = "Base path for the target hoodie dataset") final String compactionInstantTime,
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,
@@ -212,7 +212,7 @@ public class CompactionCommand implements CommandMarker {
@CliOption(key = "sparkMemory", unspecifiedDefaultValue = "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 dataset") String compactionInstantTime,
@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",
@@ -471,7 +471,7 @@ public class CompactionCommand implements CommandMarker {
if (result.get()) {
System.out.println("All renames successfully completed to " + operation + " done !!");
} else {
System.out.println("Some renames failed. DataSet could be in inconsistent-state. Try running compaction repair");
System.out.println("Some renames failed. table could be in inconsistent-state. Try running compaction repair");
}
List<Comparable[]> rows = new ArrayList<>();

View File

@@ -35,18 +35,18 @@ import org.springframework.stereotype.Component;
import scala.collection.JavaConverters;
/**
* CLI command for importing parquet dataset to hudi dataset.
* CLI command for importing parquet table to hudi table.
*/
@Component
public class HDFSParquetImportCommand implements CommandMarker {
@CliCommand(value = "hdfsparquetimport", help = "Imports Parquet dataset to a hoodie dataset")
@CliCommand(value = "hdfsparquetimport", help = "Imports Parquet table to a hoodie table")
public String convert(
@CliOption(key = "upsert", unspecifiedDefaultValue = "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 dataset") final String srcPath,
@CliOption(key = "srcPath", mandatory = true, help = "Base path for the input table") final String srcPath,
@CliOption(key = "targetPath", mandatory = true,
help = "Base path for the target hoodie dataset") final String 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,
@@ -85,8 +85,8 @@ public class HDFSParquetImportCommand implements CommandMarker {
InputStreamConsumer.captureOutput(process);
int exitCode = process.waitFor();
if (exitCode != 0) {
return "Failed to import dataset to hoodie format";
return "Failed to import table to hoodie format";
}
return "Dataset imported to hoodie format";
return "Table imported to hoodie format";
}
}

View File

@@ -65,7 +65,7 @@ public class RepairsCommand implements CommandMarker {
return "Deduplication failed ";
}
@CliCommand(value = "repair addpartitionmeta", help = "Add partition metadata to a dataset, if not present")
@CliCommand(value = "repair addpartitionmeta", help = "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)

View File

@@ -24,7 +24,7 @@ import org.apache.hudi.cli.TableHeader;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.util.ConsistencyGuardConfig;
import org.apache.hudi.exception.DatasetNotFoundException;
import org.apache.hudi.exception.TableNotFoundException;
import org.springframework.shell.core.CommandMarker;
import org.springframework.shell.core.annotation.CliCommand;
@@ -37,18 +37,18 @@ import java.util.HashMap;
import java.util.List;
/**
* CLI command to display hudi dataset options.
* CLI command to display hudi table options.
*/
@Component
public class DatasetsCommand implements CommandMarker {
public class TableCommand implements CommandMarker {
static {
System.out.println("DatasetsCommand getting loaded");
System.out.println("Table command getting loaded");
}
@CliCommand(value = "connect", help = "Connect to a hoodie dataset")
@CliCommand(value = "connect", help = "Connect to a hoodie table")
public String connect(
@CliOption(key = {"path"}, mandatory = true, help = "Base Path of the dataset") final String path,
@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",
help = "Enable eventual consistency") final boolean eventuallyConsistent,
@@ -67,7 +67,7 @@ public class DatasetsCommand implements CommandMarker {
HoodieCLI.initConf();
HoodieCLI.connectTo(path, layoutVersion);
HoodieCLI.initFS(true);
HoodieCLI.state = HoodieCLI.CLIState.DATASET;
HoodieCLI.state = HoodieCLI.CLIState.TABLE;
return "Metadata for table " + HoodieCLI.getTableMetaClient().getTableConfig().getTableName() + " loaded";
}
@@ -81,7 +81,7 @@ public class DatasetsCommand implements CommandMarker {
*/
@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 = {"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",
help = "Hoodie Table Type. Must be one of : COPY_ON_WRITE or MERGE_ON_READ") final String tableTypeStr,
@@ -98,13 +98,13 @@ public class DatasetsCommand implements CommandMarker {
try {
new HoodieTableMetaClient(HoodieCLI.conf, path);
existing = true;
} catch (DatasetNotFoundException dfe) {
} catch (TableNotFoundException dfe) {
// expected
}
// Do not touch table that already exist
if (existing) {
throw new IllegalStateException("Dataset already existing in path : " + path);
throw new IllegalStateException("Table already existing in path : " + path);
}
final HoodieTableType tableType = HoodieTableType.valueOf(tableTypeStr);