public LifecycleTransaction tryModify(SSTableReader sstable, OperationType operationType) { return tryModify(singleton(sstable), operationType); }
/** * construct a Transaction for use in an offline operation */ public static LifecycleTransaction offline(OperationType operationType, Iterable<SSTableReader> readers) { // if offline, for simplicity we just use a dummy tracker Tracker dummy = new Tracker(null, false); dummy.addInitialSSTables(readers); dummy.apply(updateCompacting(emptySet(), readers)); return new LifecycleTransaction(dummy, operationType, readers); }
public void replaceFlushed(Memtable memtable, Iterable<SSTableReader> sstables) { assert !isDummy(); if (Iterables.isEmpty(sstables)) { // sstable may be null if we flushed batchlog and nothing needed to be retained // if it's null, we don't care what state the cfstore is in, we just replace it and continue apply(View.replaceFlushed(memtable, null)); return; } sstables.forEach(SSTableReader::setupOnline); // back up before creating a new Snapshot (which makes the new one eligible for compaction) maybeIncrementallyBackup(sstables); apply(View.replaceFlushed(memtable, sstables)); Throwable fail; fail = updateSizeTracking(emptySet(), sstables, null); notifyDiscarded(memtable); // TODO: if we're invalidated, should we notifyadded AND removed, or just skip both? fail = notifyAdded(sstables, fail); if (!isDummy() && !cfstore.isValid()) dropSSTables(); maybeFail(fail); }
public void addSSTables(Iterable<SSTableReader> sstables) { addInitialSSTables(sstables); maybeIncrementallyBackup(sstables); notifyAdded(sstables); }
public Throwable dropSSTablesIfInvalid(Throwable accumulate) { if (!isDummy() && !cfstore.isValid()) accumulate = dropSSTables(accumulate); return accumulate; }
public void addInitialSSTables(Iterable<SSTableReader> sstables) { if (!isDummy()) setupOnline(sstables); apply(updateLiveSet(emptySet(), sstables)); maybeFail(updateSizeTracking(emptySet(), sstables, null)); // no notifications or backup necessary }
Pair<View, View> result = apply(view -> { Set<SSTableReader> toremove = copyOf(filter(view.sstables, and(remove, notIn(view.compacting)))); return updateLiveSet(toremove, emptySet()).apply(view); }); accumulate = updateSizeTracking(removed, emptySet(), accumulate); accumulate = release(selfRefs(removed), accumulate); accumulate = notifySSTablesChanged(removed, Collections.<SSTableReader>emptySet(), txnLogs.type(), accumulate);
Pair<View, View> result = apply(view -> { Set<SSTableReader> toremove = copyOf(filter(view.sstables, and(remove, notIn(view.compacting)))); return updateLiveSet(toremove, emptySet()).apply(view); }); accumulate = updateSizeTracking(removed, emptySet(), accumulate); accumulate = release(selfRefs(removed), accumulate);
private Throwable unmarkCompacting(Set<SSTableReader> unmark, Throwable accumulate) { accumulate = tracker.apply(updateCompacting(unmark, emptySet()), accumulate); // when the CFS is invalidated, it will call unreferenceSSTables(). However, unreferenceSSTables only deals // with sstables that aren't currently being compacted. If there are ongoing compactions that finish or are // interrupted after the CFS is invalidated, those sstables need to be unreferenced as well, so we do that here. accumulate = tracker.dropSSTablesIfInvalid(accumulate); return accumulate; }
public Throwable dropSSTables(Throwable accumulate) { return dropSSTables(Predicates.<SSTableReader>alwaysTrue(), OperationType.UNKNOWN, accumulate); }
public LifecycleTransaction call() { assert data.getCompacting().isEmpty() : data.getCompacting(); Iterable<SSTableReader> sstables = getLiveSSTables(); sstables = AbstractCompactionStrategy.filterSuspectSSTables(sstables); sstables = ImmutableList.copyOf(sstables); LifecycleTransaction modifier = data.tryModify(sstables, operationType); assert modifier != null: "something marked things compacting while compactions are disabled"; return modifier; } };
if (DatabaseDescriptor.isDaemonInitialized()) initialMemtable = new Memtable(new AtomicReference<>(CommitLog.instance.getCurrentPosition()), this); data = new Tracker(initialMemtable, loadSSTables); data.addInitialSSTables(sstables);
public boolean isOffline() { return tracker.isDummy(); }
/** * point of no return: commit all changes, but leave all readers marked as compacting */ public Throwable doCommit(Throwable accumulate) { assert staged.isEmpty() : "must be no actions introduced between prepareToCommit and a commit"; if (logger.isTraceEnabled()) logger.trace("Committing transaction over {} staged: {}, logged: {}", originals, staged, logged); // accumulate must be null if we have been used correctly, so fail immediately if it is not maybeFail(accumulate); // transaction log commit failure means we must abort; safe commit is not possible maybeFail(log.commit(null)); // this is now the point of no return; we cannot safely rollback, so we ignore exceptions until we're done // we restore state by obsoleting our obsolete files, releasing our references to them, and updating our size // and notification status for the obsolete and new files accumulate = markObsolete(obsoletions, accumulate); accumulate = tracker.updateSizeTracking(logged.obsolete, logged.update, accumulate); accumulate = release(selfRefs(logged.obsolete), accumulate); accumulate = tracker.notifySSTablesChanged(originals, logged.update, log.type(), accumulate); return accumulate; }
/** * construct an empty Transaction with no existing readers */ @SuppressWarnings("resource") // log closed during postCleanup public static LifecycleTransaction offline(OperationType operationType) { Tracker dummy = new Tracker(null, false); return new LifecycleTransaction(dummy, new LogTransaction(operationType, dummy), Collections.emptyList()); }
public void addSSTables(Collection<SSTableReader> sstables) { data.addSSTables(sstables); CompactionManager.instance.submitBackground(this); }
Throwable apply(Function<View, View> function, Throwable accumulate) { try { apply(function); } catch (Throwable t) { accumulate = merge(accumulate, t); } return accumulate; }
/** * point of no return: commit all changes, but leave all readers marked as compacting */ public Throwable doCommit(Throwable accumulate) { assert staged.isEmpty() : "must be no actions introduced between prepareToCommit and a commit"; if (logger.isTraceEnabled()) logger.trace("Committing transaction over {} staged: {}, logged: {}", originals, staged, logged); // accumulate must be null if we have been used correctly, so fail immediately if it is not maybeFail(accumulate); // transaction log commit failure means we must abort; safe commit is not possible maybeFail(log.commit(null)); // this is now the point of no return; we cannot safely rollback, so we ignore exceptions until we're done // we restore state by obsoleting our obsolete files, releasing our references to them, and updating our size // and notification status for the obsolete and new files accumulate = markObsolete(obsoletions, accumulate); accumulate = tracker.updateSizeTracking(logged.obsolete, logged.update, accumulate); accumulate = release(selfRefs(logged.obsolete), accumulate); //accumulate = tracker.notifySSTablesChanged(originals, logged.update, log.type(), accumulate); return accumulate; }
public void addInitialSSTables(Iterable<SSTableReader> sstables) { if (!isDummy()) setupOnline(sstables); apply(updateLiveSet(emptySet(), sstables)); maybeFail(updateSizeTracking(emptySet(), sstables, null)); // no notifications or backup necessary }