@Override public String getPrompt() { if (HoodieCLI.tableMetadata != null) { String tableName = HoodieCLI.tableMetadata.getTableConfig().getTableName(); switch (HoodieCLI.state) { case INIT: return "hoodie->"; case DATASET: return "hoodie:" + tableName + "->"; case SYNC: return "hoodie:" + tableName + " <==> " + HoodieCLI.syncTableMetadata.getTableConfig().getTableName() + "->"; default: return "hoodie:" + tableName + "->"; } } return "hoodie->"; }
private static long countRecords(String jdbcUrl, HoodieTableMetaClient source, String srcDb, String startDateStr, String endDateStr, String user, String pass) throws SQLException { Connection conn = HiveUtil.getConnection(jdbcUrl, user, pass); ResultSet rs = null; Statement stmt = conn.createStatement(); try { //stmt.execute("set mapred.job.queue.name=<queue_name>"); stmt.execute("set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat"); stmt.execute("set hive.stats.autogather=false"); rs = stmt.executeQuery( "select count(`_hoodie_commit_time`) as cnt from " + srcDb + "." + source.getTableConfig().getTableName() + " where datestr>'" + startDateStr + "' and datestr<='" + endDateStr + "'"); if (rs.next()) { return rs.getLong("cnt"); } return -1; } finally { if (rs != null) { rs.close(); } if (stmt != null) { stmt.close(); } } } }
public static long countRecords(String jdbcUrl, HoodieTableMetaClient source, String dbName, String user, String pass) throws SQLException { Connection conn = HiveUtil.getConnection(jdbcUrl, user, pass); ResultSet rs = null; Statement stmt = conn.createStatement(); try { //stmt.execute("set mapred.job.queue.name=<queue_name>"); stmt.execute("set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat"); stmt.execute("set hive.stats.autogather=false"); rs = stmt.executeQuery( "select count(`_hoodie_commit_time`) as cnt from " + dbName + "." + source.getTableConfig().getTableName()); long count = -1; if (rs.next()) { count = rs.getLong("cnt"); } System.out.println("Total records in " + source.getTableConfig().getTableName() + " is " + count); return count; } finally { if (rs != null) { rs.close(); } if (stmt != null) { stmt.close(); } } }
@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) throws Exception { HoodieCLI.syncTableMetadata = new HoodieTableMetaClient(HoodieCLI.conf, path); HoodieCLI.state = HoodieCLI.CLIState.SYNC; return "Load sync state between " + HoodieCLI.tableMetadata.getTableConfig().getTableName() + " and " + HoodieCLI.syncTableMetadata.getTableConfig().getTableName(); }
LOG.debug("Hoodie Metadata initialized with completed commit Ts as :" + metadata); String tableName = metadata.getTableConfig().getTableName(); String mode = HoodieHiveUtil.readMode(Job.getInstance(job), tableName);
@CliCommand(value = "savepoints refresh", help = "Refresh the savepoints") public String refreshMetaClient() throws IOException { HoodieTableMetaClient metadata = new HoodieTableMetaClient(HoodieCLI.conf, HoodieCLI.tableMetadata.getBasePath()); HoodieCLI.setTableMetadata(metadata); return "Metadata for table " + metadata.getTableConfig().getTableName() + " refreshed."; }
@CliCommand(value = "commits refresh", help = "Refresh the commits") public String refreshCommits() throws IOException { HoodieTableMetaClient metadata = new HoodieTableMetaClient(HoodieCLI.conf, HoodieCLI.tableMetadata.getBasePath()); HoodieCLI.setTableMetadata(metadata); return "Metadata for table " + metadata.getTableConfig().getTableName() + " refreshed."; }
@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) throws Exception { HoodieTableMetaClient target = new HoodieTableMetaClient(HoodieCLI.conf, path); HoodieTimeline targetTimeline = target.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); HoodieTableMetaClient source = HoodieCLI.tableMetadata; HoodieTimeline sourceTimeline = source.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); String targetLatestCommit = targetTimeline.getInstants().iterator().hasNext() ? "0" : targetTimeline.lastInstant().get().getTimestamp(); String sourceLatestCommit = sourceTimeline.getInstants().iterator().hasNext() ? "0" : sourceTimeline.lastInstant().get().getTimestamp(); if (sourceLatestCommit != null && HoodieTimeline.compareTimestamps(targetLatestCommit, sourceLatestCommit, HoodieTimeline.GREATER)) { // source is behind the target List<String> commitsToCatchup = targetTimeline.findInstantsAfter(sourceLatestCommit, Integer.MAX_VALUE) .getInstants().map(HoodieInstant::getTimestamp).collect(Collectors.toList()); return "Source " + source.getTableConfig().getTableName() + " is behind by " + commitsToCatchup.size() + " commits. Commits to catch up - " + commitsToCatchup; } else { List<String> commitsToCatchup = sourceTimeline.findInstantsAfter(targetLatestCommit, Integer.MAX_VALUE) .getInstants().map(HoodieInstant::getTimestamp).collect(Collectors.toList()); return "Source " + source.getTableConfig().getTableName() + " is ahead by " + commitsToCatchup.size() + " commits. Commits to catch up - " + commitsToCatchup; } }
@CliCommand(value = "cleans refresh", help = "Refresh the commits") public String refreshCleans() throws IOException { HoodieTableMetaClient metadata = new HoodieTableMetaClient(HoodieCLI.conf, HoodieCLI.tableMetadata.getBasePath()); HoodieCLI.setTableMetadata(metadata); return "Metadata for table " + metadata.getTableConfig().getTableName() + " refreshed."; }
.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; .getInstants().collect(Collectors.toList()); if (commitsToCatchup.isEmpty()) { return "Count difference now is (count(" + source.getTableConfig().getTableName() + ") - count(" + target.getTableConfig().getTableName() + ") == " + (sourceCount - targetCount); } else { long newInserts = CommitUtil.countNewRecords(source, commitsToCatchup.stream().map(HoodieInstant::getTimestamp).collect(Collectors.toList())); return "Count difference now is (count(" + source.getTableConfig().getTableName() + ") - count(" + target.getTableConfig().getTableName() + ") == " + (sourceCount - targetCount) + ". Catch up count is " + newInserts;
@CliCommand(value = "connect", help = "Connect to a hoodie dataset") public String connect( @CliOption(key = {"path"}, mandatory = true, help = "Base Path of the dataset") final String path) throws IOException { boolean initialized = HoodieCLI.initConf(); HoodieCLI.initFS(initialized); HoodieCLI.setTableMetadata(new HoodieTableMetaClient(HoodieCLI.conf, path)); HoodieCLI.state = HoodieCLI.CLIState.DATASET; return "Metadata for table " + HoodieCLI.tableMetadata.getTableConfig().getTableName() + " loaded"; }
@CliCommand(value = "compaction schedule", help = "Schedule Compaction") public String scheduleCompact( @CliOption(key = "sparkMemory", unspecifiedDefaultValue = "1G", help = "Spark executor memory") final String sparkMemory) throws Exception { boolean initialized = HoodieCLI.initConf(); HoodieCLI.initFS(initialized); // First get a compaction instant time and pass it to spark launcher for scheduling compaction String compactionInstantTime = HoodieActiveTimeline.createNewCommitTime(); if (HoodieCLI.tableMetadata.getTableType() == HoodieTableType.MERGE_ON_READ) { String sparkPropertiesPath = Utils.getDefaultPropertiesFile( scala.collection.JavaConversions.propertiesAsScalaMap(System.getProperties())); SparkLauncher sparkLauncher = SparkUtil.initLauncher(sparkPropertiesPath); sparkLauncher.addAppArgs(SparkCommand.COMPACT_SCHEDULE.toString(), HoodieCLI.tableMetadata.getBasePath(), HoodieCLI.tableMetadata.getTableConfig().getTableName(), compactionInstantTime, sparkMemory); Process process = sparkLauncher.launch(); InputStreamConsumer.captureOutput(process); int exitCode = process.waitFor(); if (exitCode != 0) { return "Failed to run compaction for " + compactionInstantTime; } return "Compaction successfully completed for " + compactionInstantTime; } else { throw new Exception("Compactions can only be run for table type : MERGE_ON_READ"); } }
SparkLauncher sparkLauncher = SparkUtil.initLauncher(sparkPropertiesPath); sparkLauncher.addAppArgs(SparkCommand.COMPACT_RUN.toString(), HoodieCLI.tableMetadata.getBasePath(), HoodieCLI.tableMetadata.getTableConfig().getTableName(), compactionInstantTime, parallelism, schemaFilePath, sparkMemory, retry); Process process = sparkLauncher.launch();
@Test public void checkMetadata() { assertEquals("Table name should be raw_trips", HoodieTestUtils.RAW_TRIPS_TEST_NAME, metaClient.getTableConfig().getTableName()); assertEquals("Basepath should be the one assigned", basePath, metaClient.getBasePath()); assertEquals("Metapath should be ${basepath}/.hoodie", basePath + "/.hoodie", metaClient.getMetaPath()); }