private Stream<HoodieDataFile> convertFileStatusesToDataFiles(FileStatus[] statuses) { Predicate<FileStatus> roFilePredicate = fileStatus -> fileStatus.getPath().getName() .contains(metaClient.getTableConfig().getROFileFormat().getFileExtension()); return Arrays.stream(statuses).filter(roFilePredicate).map(HoodieDataFile::new); }
@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 Stream<HoodieLogFile> convertFileStatusesToLogFiles(FileStatus[] statuses) { Predicate<FileStatus> rtFilePredicate = fileStatus -> fileStatus.getPath().getName() .contains(metaClient.getTableConfig().getRTFileFormat().getFileExtension()); return Arrays.stream(statuses).filter(rtFilePredicate).map(HoodieLogFile::new); }
public HoodieTableMetaClient(Configuration conf, String basePath, boolean loadActiveTimelineOnLoad) throws DatasetNotFoundException { log.info("Loading HoodieTableMetaClient from " + basePath); this.basePath = basePath; this.hadoopConf = new SerializableConfiguration(conf); Path basePathDir = new Path(this.basePath); this.metaPath = basePath + File.separator + METAFOLDER_NAME; Path metaPathDir = new Path(this.metaPath); this.fs = getFs(); DatasetNotFoundException.checkValidDataset(fs, basePathDir, metaPathDir); this.tableConfig = new HoodieTableConfig(fs, metaPath); this.tableType = tableConfig.getTableType(); log.info("Finished Loading Table of type " + tableType + " from " + basePath); if (loadActiveTimelineOnLoad) { log.info("Loading Active commit timeline for " + basePath); getActiveTimeline(); } }
HoodieTableConfig.createHoodieProperties(fs, metaPathDir, props); log.info("Finished initializing Table of type " + metaClient.getTableConfig().getTableType() + " from " + basePath); return metaClient;
private Stream<Pair<String, String>> convertPathToFileIdWithCommitTime( final HoodieTableMetaClient metaClient, List<String> paths) { Predicate<String> roFilePredicate = path -> path.contains(metaClient.getTableConfig().getROFileFormat().getFileExtension()); Predicate<String> rtFilePredicate = path -> path.contains(metaClient.getTableConfig().getRTFileFormat().getFileExtension()); Stream<Pair<String, String>> stream1 = paths.stream().filter(roFilePredicate) .map(fullPath -> { String fileName = Paths.get(fullPath).getFileName().toString(); return Pair.of(FSUtils.getFileId(fileName), FSUtils.getCommitTime(fileName)); }); Stream<Pair<String, String>> stream2 = paths.stream().filter(rtFilePredicate) .map(path -> { return Pair.of(FSUtils.getFileIdFromLogPath(new Path(path)), FSUtils.getBaseCommitTimeFromLogPath(new Path(path))); }); return Stream.concat(stream1, stream2); } }
/** * @return path where archived timeline is stored */ public String getArchivePath() { String archiveFolder = tableConfig.getArchivelogFolder(); if (archiveFolder.equals(HoodieTableConfig.DEFAULT_ARCHIVELOG_FOLDER)) { return getMetaPath(); } else { return getMetaPath() + "/" + archiveFolder; } }
public AbstractHoodieLogRecordScanner(FileSystem fs, String basePath, List<String> logFilePaths, Schema readerSchema, String latestInstantTime, boolean readBlocksLazily, boolean reverseReader, int bufferSize) { this.readerSchema = readerSchema; this.latestInstantTime = latestInstantTime; this.hoodieTableMetaClient = new HoodieTableMetaClient(fs.getConf(), basePath); // load class from the payload fully qualified class name this.payloadClassFQN = this.hoodieTableMetaClient.getTableConfig().getPayloadClass(); this.totalLogFiles.addAndGet(logFilePaths.size()); this.logFilePaths = logFilePaths; this.readBlocksLazily = readBlocksLazily; this.reverseReader = reverseReader; this.fs = fs; this.bufferSize = bufferSize; }
/** * Describes table properties */ @CliCommand(value = "desc", help = "Describle Hoodie Table properties") public String descTable() { TableHeader header = new TableHeader() .addTableHeaderField("Property") .addTableHeaderField("Value"); List<Comparable[]> 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); } }
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(); } } } }
.filter(s -> s.contains((metaClient.getTableConfig().getROFileFormat().getFileExtension()))) .findAny() .map(f -> Pair.of(f, HoodieFileFormat.PARQUET)).orElseThrow(() -> {
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 = "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 = "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 = "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"; }