/** * Provides a stream of committed file slices, sorted reverse base commit time. */ public Stream<FileSlice> getAllFileSlices() { if (!timeline.empty()) { return fileSlices.entrySet().stream() .map(Map.Entry::getValue) .filter(this::isFileSliceCommitted); } return Stream.empty(); }
@CliAvailabilityIndicator({"savepoint rollback"}) public boolean isRollbackToSavepointAvailable() { return HoodieCLI.tableMetadata != null && !HoodieCLI.tableMetadata.getActiveTimeline().getSavePointTimeline() .filterCompletedInstants().empty(); }
private boolean checkIfValidCommit(HoodieTableMetaClient metaClient, String commitTs) { HoodieTimeline commitTimeline = metaClient.getActiveTimeline().filterCompletedInstants(); // Check if the last commit ts for this row is 1) present in the timeline or // 2) is less than the first commit ts in the timeline return !commitTimeline.empty() && (commitTimeline .containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commitTs)) || HoodieTimeline .compareTimestamps(commitTimeline.firstInstant().get().getTimestamp(), commitTs, HoodieTimeline.GREATER)); }
if (commitTimeline.empty() && inflightCommitTimeline.empty()) { if ((lastCommit != null) && !commitTimeline.empty() && !commitTimeline .findInstantsAfter(lastCommit, Integer.MAX_VALUE).empty()) { throw new HoodieRollbackException( "Found commits after time :" + lastCommit + ", please rollback greater commits first"); logger.info("Commits " + commitsToRollback + " rollback is complete"); if (!table.getActiveTimeline().getCleanerTimeline().empty()) { logger.info("Cleaning up older rollback meta files");
private boolean checkIfValidCommit(HoodieTableMetaClient metaClient, String commitTs) { HoodieTimeline commitTimeline = metaClient.getActiveTimeline().filterCompletedInstants(); // Check if the last commit ts for this row is 1) present in the timeline or // 2) is less than the first commit ts in the timeline return !commitTimeline.empty() && (commitTimeline .containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commitTs)) || HoodieTimeline .compareTimestamps(commitTimeline.firstInstant().get().getTimestamp(), commitTs, HoodieTimeline.GREATER)); }
if (!commitTimeline.empty() && commitTimeline.countInstants() > maxCommitsToKeep) {
if (commitTimeline.empty() && inflightCommitTimeline.empty()) { if ((lastCommit != null) && !commitTimeline.empty() && !commitTimeline .findInstantsAfter(lastCommit, Integer.MAX_VALUE).empty()) { throw new HoodieRollbackException( "Found commits after time :" + lastCommit + ", please rollback greater commits first"); logger.info("Commits " + commitsToRollback + " rollback is complete"); if (!table.getActiveTimeline().getCleanerTimeline().empty()) { logger.info("Cleaning up older rollback meta files");
if (!commitTimeline.empty() && commitTimeline.countInstants() > maxCommitsToKeep) {
/** * Savepoint a specific commit. Latest version of data files as of the passed in commitTime will * be referenced in the savepoint and will never be cleaned. The savepointed commit will never be * rolledback or archived. * <p> * This gives an option to rollback the state to the savepoint anytime. Savepoint needs to be * manually created and deleted. * <p> * Savepoint should be on a commit that could not have been cleaned. * * @param user - User creating the savepoint * @param comment - Comment for the savepoint * @return true if the savepoint was created successfully */ public boolean savepoint(String user, String comment) { HoodieTable<T> table = HoodieTable.getHoodieTable( new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config, jsc); if (table.getCompletedCommitsTimeline().empty()) { throw new HoodieSavepointException("Could not savepoint. Commit timeline is empty"); } String latestCommit = table.getCompletedCommitsTimeline().lastInstant().get().getTimestamp(); logger.info("Savepointing latest commit " + latestCommit); return savepoint(latestCommit, user, comment); }
/** * Savepoint a specific commit. Latest version of data files as of the passed in commitTime will * be referenced in the savepoint and will never be cleaned. The savepointed commit will never be * rolledback or archived. * <p> * This gives an option to rollback the state to the savepoint anytime. Savepoint needs to be * manually created and deleted. * <p> * Savepoint should be on a commit that could not have been cleaned. * * @param user - User creating the savepoint * @param comment - Comment for the savepoint * @return true if the savepoint was created successfully */ public boolean savepoint(String user, String comment) { HoodieTable<T> table = HoodieTable.getHoodieTable( new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config, jsc); if (table.getCompletedCommitTimeline().empty()) { throw new HoodieSavepointException("Could not savepoint. Commit timeline is empty"); } String latestCommit = table.getCompletedCommitTimeline().lastInstant().get().getTimestamp(); logger.info("Savepointing latest commit " + latestCommit); return savepoint(latestCommit, user, comment); }
logger.info("Marked clean started on " + startCleanTime + " as complete"); if (!table.getActiveTimeline().getCleanerTimeline().empty()) {
logger.info("Marked clean started on " + startCleanTime + " as complete"); if (!table.getActiveTimeline().getCleanerTimeline().empty()) {
@Test public void checkCommitTimeline() throws IOException { HoodieActiveTimeline activeTimeline = metaClient.getActiveTimeline(); HoodieTimeline activeCommitTimeline = activeTimeline.getCommitTimeline(); assertTrue("Should be empty commit timeline", activeCommitTimeline.empty()); HoodieInstant instant = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, "1"); activeTimeline.createInflight(instant); activeTimeline.saveAsComplete(instant, Optional.of("test-detail".getBytes())); // Commit timeline should not auto-reload every time getActiveCommitTimeline(), it should be cached activeTimeline = metaClient.getActiveTimeline(); activeCommitTimeline = activeTimeline.getCommitTimeline(); assertTrue("Should be empty commit timeline", activeCommitTimeline.empty()); HoodieInstant completedInstant = HoodieTimeline.getCompletedInstant(instant); activeTimeline = activeTimeline.reload(); activeCommitTimeline = activeTimeline.getCommitTimeline(); assertFalse("Should be the 1 commit we made", activeCommitTimeline.empty()); assertEquals("Commit should be 1", completedInstant, activeCommitTimeline.getInstants().findFirst().get()); assertArrayEquals("Commit value should be \"test-detail\"", "test-detail".getBytes(), activeCommitTimeline.getInstantDetails(completedInstant).get()); }
new HoodieTableMetaClient( new HadoopConfiguration(hoodieConf.getConf()).getHadoopConf(), basePath.toString(), true) .getActiveTimeline().getCommitTimeline().filterCompletedInstants().empty()); new HoodieTableMetaClient( new HadoopConfiguration(hoodieConf.getConf()).getHadoopConf(), basePath.toString(), true) .getActiveTimeline().getCommitTimeline().filterCompletedInstants().empty());
@Test public void testTimelineOperations() throws Exception { timeline = new MockHoodieTimeline(Stream.of("01", "03", "05", "07", "09", "11", "13", "15", "17", "19"), Stream.of("21", "23")); HoodieTestUtils.assertStreamEquals("", Stream.of("05", "07", "09", "11"), timeline.getCommitTimeline().filterCompletedInstants().findInstantsInRange("04", "11").getInstants() .map(HoodieInstant::getTimestamp)); HoodieTestUtils.assertStreamEquals("", Stream.of("09", "11"), timeline.getCommitTimeline().filterCompletedInstants().findInstantsAfter("07", 2).getInstants() .map(HoodieInstant::getTimestamp)); assertFalse(timeline.empty()); assertFalse(timeline.getCommitTimeline().filterInflightsExcludingCompaction().empty()); assertEquals("", 12, timeline.countInstants()); HoodieTimeline activeCommitTimeline = timeline.getCommitTimeline().filterCompletedInstants(); assertEquals("", 10, activeCommitTimeline.countInstants()); assertEquals("", "01", activeCommitTimeline.firstInstant().get().getTimestamp()); assertEquals("", "11", activeCommitTimeline.nthInstant(5).get().getTimestamp()); assertEquals("", "19", activeCommitTimeline.lastInstant().get().getTimestamp()); assertEquals("", "09", activeCommitTimeline.nthFromLastInstant(5).get().getTimestamp()); assertTrue("", activeCommitTimeline.containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "09"))); assertFalse("", activeCommitTimeline.isBeforeTimelineStarts("02")); assertTrue("", activeCommitTimeline.isBeforeTimelineStarts("00")); } }
assertFalse(table.getCompletedCommitsTimeline().empty()); String commitTime = table.getCompletedCommitsTimeline().getInstants().findFirst().get().getTimestamp(); assertFalse(table.getCompletedCleanTimeline().empty()); assertEquals("The clean instant should be the same as the commit instant", commitTime, table.getCompletedCleanTimeline().getInstants().findFirst().get().getTimestamp());