public Builder withEarliestCommitRetained(Optional<HoodieInstant> earliestCommitToRetain) { this.earliestCommitToRetain = (earliestCommitToRetain.isPresent()) ? earliestCommitToRetain.get().getTimestamp() : "-1"; return this; }
/** * Transition Compaction State from inflight to Committed * * @param inflightInstant Inflight instant * @param data Extra Metadata * @return commit instant */ public HoodieInstant transitionCompactionInflightToComplete(HoodieInstant inflightInstant, Optional<byte[]> data) { Preconditions.checkArgument(inflightInstant.getAction().equals(HoodieTimeline.COMPACTION_ACTION)); Preconditions.checkArgument(inflightInstant.isInflight()); HoodieInstant commitInstant = new HoodieInstant(State.COMPLETED, COMMIT_ACTION, inflightInstant.getTimestamp()); transitionState(inflightInstant, commitInstant, data); return commitInstant; }
private void revertCompleteToInflight(HoodieInstant completed, HoodieInstant inflight) { Preconditions.checkArgument(completed.getTimestamp().equals(inflight.getTimestamp())); Path inFlightCommitFilePath = new Path(metaClient.getMetaPath(), inflight.getFileName()); try { if (!metaClient.getFs().exists(inFlightCommitFilePath)) { Path commitFilePath = new Path(metaClient.getMetaPath(), completed.getFileName()); boolean success = metaClient.getFs().rename(commitFilePath, inFlightCommitFilePath); if (!success) { throw new HoodieIOException( "Could not rename " + commitFilePath + " to " + inFlightCommitFilePath); } } } catch (IOException e) { throw new HoodieIOException("Could not complete revert " + completed, e); } }
@Override public String toString() { return "[" + ((isInflight() || isRequested()) ? "==>" : "") + timestamp + "__" + action + "__" + state + "]"; } }
static HoodieInstant getCompletedInstant(final HoodieInstant instant) { return new HoodieInstant(false, instant.getAction(), instant.getTimestamp()); }
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)); }
/** * Transition Compaction State from requested to inflight * * @param requestedInstant Requested instant * @return inflight instant */ public HoodieInstant transitionCompactionRequestedToInflight(HoodieInstant requestedInstant) { Preconditions.checkArgument(requestedInstant.getAction().equals(HoodieTimeline.COMPACTION_ACTION)); Preconditions.checkArgument(requestedInstant.isRequested()); HoodieInstant inflightInstant = new HoodieInstant(State.INFLIGHT, COMPACTION_ACTION, requestedInstant.getTimestamp()); transitionState(requestedInstant, inflightInstant, Optional.empty()); return inflightInstant; }
.filterCompletedInstants(); Stream<HoodieInstant> instants = cleanAndRollbackTimeline.getInstants() .collect(Collectors.groupingBy(s -> s.getAction())).entrySet().stream().map(i -> { if (i.getValue().size() > maxCommitsToKeep) { return i.getValue().subList(0, i.getValue().size() - minCommitsToKeep); .compareTimestamps(firstSavepoint.get().getTimestamp(), s.getTimestamp(), HoodieTimeline.LESSER_OR_EQUAL)); }) return HoodieTimeline.compareTimestamps(instant.getTimestamp(), s.getTimestamp(), HoodieTimeline.GREATER); }).orElse(true); })
static HoodieInstant getCompactionRequestedInstant(final String timestamp) { return new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, timestamp); }
HoodieInstant instant = instants.get(i); HoodieCompactionPlan workload = null; if (!instant.getAction().equals(COMPACTION_ACTION)) { try { HoodieTimeline.getCompactionRequestedInstant(instant.getTimestamp())).get()); } catch (HoodieIOException ioe) { HoodieTimeline.getCompactionRequestedInstant(instant.getTimestamp())).get()); HoodieInstant.State state = instant.getState(); if (committed.contains(instant.getTimestamp())) { state = State.COMPLETED; rows.add(new Comparable[]{instant.getTimestamp(), state.toString(), workload.getOperations() == null ? 0 : workload.getOperations().size(), workload.getExtraMetadata().toString()}); } else { rows.add(new Comparable[]{instant.getTimestamp(), state.toString(), workload.getOperations() == null ? 0 : workload.getOperations().size()});
private void moveCompactionFromRequestedToInflight(String compactionInstantTime, HoodieWriteClient client, HoodieWriteConfig cfg) throws IOException { HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath()); HoodieInstant compactionInstant = HoodieTimeline.getCompactionRequestedInstant(compactionInstantTime); HoodieCompactionPlan workload = AvroUtils.deserializeCompactionPlan( metaClient.getActiveTimeline().getInstantAuxiliaryDetails(compactionInstant).get()); metaClient.getActiveTimeline().transitionCompactionRequestedToInflight(compactionInstant); HoodieInstant instant = metaClient.getActiveTimeline().reload().filterPendingCompactionTimeline().getInstants() .filter(in -> in.getTimestamp().equals(compactionInstantTime)).findAny().get(); assertTrue("Instant must be marked inflight", instant.isInflight()); }
HoodieCompactionPlan.newBuilder().setOperations(new ArrayList<>()).setExtraMetadata(plan.getExtraMetadata()) .build(); HoodieInstant inflight = new HoodieInstant(State.INFLIGHT, COMPACTION_ACTION, compactionInstant); Path inflightPath = new Path(metaClient.getMetaPath(), inflight.getFileName()); if (metaClient.getFs().exists(inflightPath)) { new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, compactionInstant), AvroUtils.serializeCompactionPlan(newPlan));
public static void deleteOlderCleanMetaFiles(FileSystem fs, String metaPath, Stream<HoodieInstant> instants) { //TODO - this should be archived when archival is made general for all meta-data // skip MIN_CLEAN_TO_KEEP and delete rest instants.skip(MIN_CLEAN_TO_KEEP).map(s -> { try { return fs.delete(new Path(metaPath, s.getFileName()), false); } catch (IOException e) { throw new HoodieIOException("Could not delete clean meta files" + s.getFileName(), e); } }); }
private boolean deleteArchivedInstants(List<HoodieInstant> archivedInstants) throws IOException { log.info("Deleting instants " + archivedInstants); boolean success = true; for (HoodieInstant archivedInstant : archivedInstants) { Path commitFile = new Path(metaClient.getMetaPath(), archivedInstant.getFileName()); try { if (metaClient.getFs().exists(commitFile)) { success &= metaClient.getFs().delete(commitFile, false); log.info("Archived and deleted instant file " + commitFile); } } catch (IOException e) { throw new HoodieIOException("Failed to delete archived instant " + archivedInstant, e); } } // Remove older meta-data from auxiliary path too Optional<HoodieInstant> latestCommitted = archivedInstants.stream() .filter(i -> { return i.isCompleted() && (i.getAction().equals(COMMIT_ACTION) || (i.getAction().equals(DELTA_COMMIT_ACTION))); }).max(Comparator.comparing(HoodieInstant::getTimestamp)); if (latestCommitted.isPresent()) { success &= deleteAllInstantsOlderorEqualsInAuxMetaFolder(latestCommitted.get()); } return success; }
public void saveToInflight(HoodieInstant instant, Optional<byte[]> content) { Preconditions.checkArgument(instant.isInflight()); createFileInMetaPath(instant.getFileName(), content); }
public void saveToCompactionRequested(HoodieInstant instant, Optional<byte[]> content) { Preconditions.checkArgument(instant.getAction().equals(HoodieTimeline.COMPACTION_ACTION)); // Write workload to auxiliary folder createFileInAuxiliaryFolder(instant, content); createFileInMetaPath(instant.getFileName(), content); }
@Override public HoodieTimeline filterCompletedAndCompactionInstants() { return new HoodieDefaultTimeline(instants.stream().filter(s -> { return !s.isInflight() || s.getAction().equals(HoodieTimeline.COMPACTION_ACTION); }), details); }
public void deleteCompactionRequested(HoodieInstant instant) { Preconditions.checkArgument(instant.isRequested()); Preconditions.checkArgument(instant.getAction() == HoodieTimeline.COMPACTION_ACTION); deleteInstantFile(instant); }
protected Stream<HoodieInstant> filterInstantsByAction(String action) { return instants.stream().filter(s -> s.getAction().equals(action)); }
@Override public HoodieTimeline filterCompletedInstants() { return new HoodieDefaultTimeline(instants.stream().filter(s -> !s.isInflight()), details); }