-
Notifications
You must be signed in to change notification settings - Fork 3.8k
Replace blocking wait with non-blocking delay in paxos repair #4434
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
base: trunk
Are you sure you want to change the base?
Changes from 1 commit
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -20,14 +20,17 @@ | |
|
|
||
| import java.util.Collection; | ||
| import java.util.Map; | ||
| import java.util.Queue; | ||
| import java.util.UUID; | ||
| import java.util.concurrent.ConcurrentHashMap; | ||
| import java.util.concurrent.LinkedBlockingQueue; | ||
|
|
||
| import com.google.common.base.Preconditions; | ||
| import com.google.common.util.concurrent.Uninterruptibles; | ||
| import org.slf4j.Logger; | ||
| import org.slf4j.LoggerFactory; | ||
|
|
||
| import org.apache.cassandra.concurrent.ScheduledExecutors; | ||
| import org.apache.cassandra.config.DatabaseDescriptor; | ||
| import org.apache.cassandra.db.ConsistencyLevel; | ||
| import org.apache.cassandra.db.DecoratedKey; | ||
|
|
@@ -58,6 +61,18 @@ public class PaxosCleanupLocalCoordinator extends AsyncFuture<PaxosCleanupRespon | |
| private static final Logger logger = LoggerFactory.getLogger(PaxosCleanupLocalCoordinator.class); | ||
| private static final UUID INTERNAL_SESSION = new UUID(0, 0); | ||
|
|
||
| private static class DelayedRepair | ||
| { | ||
| private final UncommittedPaxosKey uncommitted; | ||
| private final long startAfterMillis; | ||
|
|
||
| public DelayedRepair(UncommittedPaxosKey uncommitted, long startAfterMillis) | ||
| { | ||
| this.uncommitted = uncommitted; | ||
| this.startAfterMillis = startAfterMillis; | ||
| } | ||
| } | ||
|
|
||
| private final UUID session; | ||
| private final TableId tableId; | ||
| private final TableMetadata table; | ||
|
|
@@ -69,6 +84,7 @@ public class PaxosCleanupLocalCoordinator extends AsyncFuture<PaxosCleanupRespon | |
| private final boolean autoRepair; | ||
|
|
||
| private final Map<DecoratedKey, AbstractPaxosRepair> inflight = new ConcurrentHashMap<>(); | ||
| private final Queue<DelayedRepair> delayed = new LinkedBlockingQueue<>(); | ||
|
||
| private final PaxosTableRepairs tableRepairs; | ||
|
|
||
| private PaxosCleanupLocalCoordinator(SharedContext ctx, UUID session, TableId tableId, Collection<Range<Token>> ranges, CloseableIterator<UncommittedPaxosKey> uncommittedIter, boolean autoRepair) | ||
|
|
@@ -125,6 +141,31 @@ public static PaxosCleanupLocalCoordinator createForAutoRepair(SharedContext ctx | |
| return new PaxosCleanupLocalCoordinator(ctx, INTERNAL_SESSION, tableId, ranges, iterator, true); | ||
| } | ||
|
|
||
| private boolean maybeDelay(UncommittedPaxosKey uncommitted) | ||
| { | ||
| if (!DatabaseDescriptor.getPaxosRepairRaceWait()) | ||
| return false; | ||
|
|
||
|
|
||
| long txnTimeoutMillis = Math.max(getCasContentionTimeout(MILLISECONDS), getWriteRpcTimeout(MILLISECONDS)); | ||
| long nowMillis = Clock.Global.currentTimeMillis(); | ||
| long ballotElapsedMillis = nowMillis - MICROSECONDS.toMillis(uncommitted.ballot().unixMicros()); | ||
|
|
||
| if (ballotElapsedMillis < 0 && Math.abs(ballotElapsedMillis) > SECONDS.toMillis(1)) | ||
| logger.warn("Encountered ballot that is more than 1 second in the future, is there a clock sync issue? {}", uncommitted.ballot()); | ||
|
|
||
| if (ballotElapsedMillis >= txnTimeoutMillis) | ||
| return false; | ||
|
|
||
| long sleepMillis = txnTimeoutMillis - ballotElapsedMillis; | ||
| logger.info("Paxos auto repair encountered a potentially in progress ballot, sleeping {}ms to allow the in flight operation to finish", sleepMillis); | ||
|
|
||
| delayed.add(new DelayedRepair(uncommitted, nowMillis + sleepMillis)); | ||
| ScheduledExecutors.scheduledFastTasks.schedule(this::scheduleKeyRepairsOrFinish, sleepMillis, MILLISECONDS); | ||
|
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I think we need to synchronise scheduleKeyRepairsOrFinish? Or wrap it in a synchronise call inside the lambda. Currently it inherits its safety from its callers |
||
|
|
||
| return true; | ||
| } | ||
|
|
||
| /** | ||
| * Schedule as many key repairs as we can, up to the paralellism limit. If no repairs are scheduled and | ||
| * none are in flight when the iterator is exhausted, the session will be finished | ||
|
|
@@ -141,18 +182,33 @@ private void scheduleKeyRepairsOrFinish() | |
| return; | ||
| } | ||
|
|
||
| long txnTimeoutMicros = Math.max(getCasContentionTimeout(MICROSECONDS), getWriteRpcTimeout(MICROSECONDS)); | ||
| boolean waitForCoordinator = DatabaseDescriptor.getPaxosRepairRaceWait(); | ||
| while (inflight.size() < parallelism && uncommittedIter.hasNext()) | ||
| repairKey(uncommittedIter.next(), txnTimeoutMicros, waitForCoordinator); | ||
|
|
||
| while (inflight.size() < parallelism && !isDone()) | ||
| { | ||
| if (!delayed.isEmpty() && delayed.peek().startAfterMillis < Clock.Global.currentTimeMillis()) | ||
| { | ||
| DelayedRepair delayedRepair = delayed.remove(); | ||
| repairKey(delayedRepair.uncommitted); | ||
| } | ||
| else if (uncommittedIter.hasNext()) | ||
| { | ||
| UncommittedPaxosKey uncommitted = uncommittedIter.next(); | ||
| if (!maybeDelay(uncommitted)) | ||
| { | ||
| repairKey(uncommitted); | ||
| } | ||
| } | ||
| else | ||
| { | ||
| break; | ||
| } | ||
| } | ||
| } | ||
|
|
||
| if (inflight.isEmpty()) | ||
| finish(); | ||
| } | ||
|
|
||
| private boolean repairKey(UncommittedPaxosKey uncommitted, long txnTimeoutMicros, boolean waitForCoordinator) | ||
| private boolean repairKey(UncommittedPaxosKey uncommitted) | ||
| { | ||
| logger.trace("repairing {}", uncommitted); | ||
| Preconditions.checkState(!inflight.containsKey(uncommitted.getKey())); | ||
|
|
@@ -163,9 +219,6 @@ private boolean repairKey(UncommittedPaxosKey uncommitted, long txnTimeoutMicros | |
| if (consistency == null) | ||
| return false; | ||
|
|
||
| if (waitForCoordinator) | ||
| maybeWaitForOriginalCoordinator(uncommitted, txnTimeoutMicros); | ||
|
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Can we also delete |
||
|
|
||
| inflight.put(uncommitted.getKey(), tableRepairs.startOrGetOrQueue(uncommitted.getKey(), uncommitted.ballot(), uncommitted.getConsistencyLevel(), table, result -> { | ||
| if (result.wasSuccessful()) | ||
| onKeyFinish(uncommitted.getKey()); | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I assumed this was a time delta when I read it (i.e. after millis (have elapsed)). Perhaps
atMillisorstartAtMillis, orscheduledAtMillis?But also, this should probably be nanos and we should probably use
nanoTime?