-
Couldn't load subscription status.
- Fork 3.8k
CASSANDRA-20383 #4428
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: cep-45-mutation-tracking
Are you sure you want to change the base?
CASSANDRA-20383 #4428
Conversation
patch by Abe Ratnofsky; reviewed by Blake Eggleston for CASSANDRA-20383
…sfer metadata, purge transfers based on reconciliation state
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.
You have some end to end tests, but this could also use some testing of the smaller components in more detail / with more cases.
I think it's ok to not have durability and GC as part of the initial implementation, but we do need follow tickets for them. Working out when and how to abandon transfers deserves its own ticket anyway
| /** | ||
| * See {@link UnreconciledMutations}. | ||
| * | ||
| * For now, all reads intersect with all transfers, but we could be more discerning and only return transfers ƒor the |
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.
how does that work with larger clusters where each node replicates it's own subset of ranges? Wouldn't every read (potentially) contain transfer ids that are for ranges that the other read participants don't replicate and would therefore not have in their own set of transfer ids?
| ILifecycleTransaction txn = createTxn(); | ||
| RangeAwareSSTableWriter writer = new RangeAwareSSTableWriter(cfs, estimatedKeys, repairedAt, pendingRepair, coordinatorLogOffsets, format, sstableLevel, totalSize, txn, getHeader(cfs.metadata())); | ||
| return new SSTableTxnSingleStreamWriter(txn, writer); | ||
| if (isTracked) |
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.
Whether the streamed sstables are cordoned off for bulk tracking or added via the normal stream path should be dictated by the senders session info and the stream init message. I don't think we want an explicit add step for bootstrapped sstables etc.
| { | ||
| try | ||
| { | ||
| sstable.mutateCoordinatorLogOffsetsAndReload(ImmutableCoordinatorLogOffsets.NONE); |
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.
if we loosen up validation on the stream receiving side, we can remove this step, which would make it safe for use with failure recovery
| } | ||
| } | ||
|
|
||
| private static class CoordinatedTransfers implements Iterable<CoordinatedTransfer> |
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 don't think this needs to be a private inner class. It could also use some unit tests checking the sstable selection logic
| { | ||
| ColumnFamilyStore.ViewFragment view = cfs.select(View.selectLive(dataRange().keyRange())); | ||
| if (cfs.metadata().replicationType().isTracked()) | ||
| controller.addActivationIds(view); |
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.
discussed elsewhere, but we should only be adding activation ids that intersect with the key/range of the read
| * <p> | ||
| * REVIEW: Reset back to 1 because for transfers, replicas need to know each others' shards, since transfers are | ||
| * sliced to fit within shards. Can we achieve sharding via split range ownership, instead of it being local-only? | ||
| * <p> |
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.
This was added as an aid to perf testing. I think reverting it to one is fine, especially since there's a functional reason here, but we should open a jira to revisit it
| LocalTransfers.instance().received(transfer); | ||
| } | ||
|
|
||
| synchronized void activateLocal(TransferActivation activation) |
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.
should this be synchronized? I don't think we need to guard against concurrent activation of different activations do we?
| cfs.getTracker().addSSTablesTracked(Collections.singleton(moved)); | ||
| } | ||
|
|
||
| activated = true; |
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.
obviously we don't have durability implemented for this yet, but we should definitely have tests validating the restart safety of partially completed activations when we do
| } | ||
|
|
||
| Preconditions.checkState(sstable.getCoordinatorLogOffsets().isEmpty()); | ||
| Preconditions.checkState(!sstable.getCoordinatorLogOffsets().transfers().isEmpty()); |
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.
this is a little weird from an api perspective. We're asserting that the log offsets are empty, then asserting that a collection contained in them is not empty. We should probably rename the ImmutableLogOffsets.isEmpty to something indicating that we want to know if the offsets are empty, or if the transfers are empty. There may also be other spots in the code where we take some action if the log offsets are empty without confirming that the transfers are also empty. So we should rethink some parts of this to make it less likely to be misused
| ImmutableCoordinatorLogOffsets.Builder builder = new ImmutableCoordinatorLogOffsets.Builder(); | ||
| for (SSTableReader sstable : sstables) | ||
| builder.addAll(sstable.getCoordinatorLogOffsets()); | ||
| builder.purgeTransfers(); |
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.
wouldn't this cause us to promote sstables tables containing transfers that aren't fully reconciled?
…d reconciliation and ignore ReadRepairStrategy NONE
No description provided.