cassandra icon indicating copy to clipboard operation
cassandra copied to clipboard

Implement ShardManagerReplicaAware class align UCS and replica shards

Open michaeljmarshall opened this issue 1 year ago • 10 comments

Implement ShardManagerReplicaAware to align UCS and replica shards and thus limit the amount of sstables that are partially owned by replicas.

The most interesting details are in the IsolatedTokenAllocator#allocateTokens and the ShardManagerReplicaAware#computeBoundaries methods.

In the allocateTokens method, we take the current token metadata for a cluster, replace the snitch with one that does not gossip, and allocate new nodes until we satisfy the desired additionalSplits needed. By using the token allocation algorithm, high level split points naturally align with replica shards as new nodes are added.

In computeBoundaries, we allocate any tokens needed, then we split the space into even spans and find the nearest replica token boundaries.

michaeljmarshall avatar Aug 30 '24 21:08 michaeljmarshall

@blambov - please take a look. This is my initial take at the ShardManagerNodeAware. I left several TODOs in the code as questions for you. I haven't had a chance to do any testing yet. I am looking to get general feedback on the direction and on my understanding of what is necessary here. Thanks.

michaeljmarshall avatar Aug 30 '24 21:08 michaeljmarshall

@blambov - do you mind taking another look? Thanks!

michaeljmarshall avatar Sep 04 '24 03:09 michaeljmarshall

Marking as ready for review. There is still work to be done, but this will let tests run.

michaeljmarshall avatar Sep 17 '24 17:09 michaeljmarshall

Could we temporarily change DEFAULT_IS_NODE_AWARE to true to run tests with it?

blambov avatar Sep 18 '24 12:09 blambov

Looks like we hit an UnsupportedOperationException in several tests. Checking to see if it's an issue.

java.lang.UnsupportedOperationException: Token type BytesToken does not support token allocation.

	at org.apache.cassandra.dht.ByteOrderedPartitioner$BytesToken.size(ByteOrderedPartitioner.java:134)
	at org.apache.cassandra.db.compaction.ShardManagerTokenAware$TokenAlignedShardTracker.rangeSpanned(ShardManagerTokenAware.java:285)
	at org.apache.cassandra.db.compaction.ShardTracker.applyTokenSpaceCoverage(ShardTracker.java:78)
	at org.apache.cassandra.db.compaction.ShardManagerTokenAware$TokenAlignedShardTracker.applyTokenSpaceCoverage(ShardManagerTokenAware.java:305)
	at org.apache.cassandra.db.compaction.unified.ShardedMultiWriter.prepareToCommit(ShardedMultiWriter.java:256)
	at org.apache.cassandra.db.ColumnFamilyStore$Flush.flushMemtable(ColumnFamilyStore.java:1406)
	at org.apache.cassandra.db.ColumnFamilyStore$Flush.run(ColumnFamilyStore.java:1315)
	at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
	at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
	at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
	at java.base/java.lang.Thread.run(Thread.java:829)

michaeljmarshall avatar Sep 18 '24 21:09 michaeljmarshall

@blambov - at this point, the remaining test failures appear to be from disk boundaries attempting to combine with the token aware shard manager:

	Caused by: java.lang.IllegalArgumentException: Cannot use node aware strategy with disk boundaries
		at org.apache.cassandra.db.compaction.ShardManager.create(ShardManager.java:51)
		at org.apache.cassandra.db.compaction.UnifiedCompactionStrategy.maybeUpdateSelector(UnifiedCompactionStrategy.java:400)
		at org.apache.cassandra.db.compaction.UnifiedCompactionStrategy.createSSTableMultiWriter(UnifiedCompactionStrategy.java:351)
		at org.apache.cassandra.db.compaction.UnifiedCompactionContainer.createSSTableMultiWriter(UnifiedCompactionContainer.java:327)
		at org.apache.cassandra.db.ColumnFamilyStore.createSSTableMultiWriter(ColumnFamilyStore.java:735)
		at org.apache.cassandra.db.ColumnFamilyStore.createSSTableMultiWriter(ColumnFamilyStore.java:730)
		at org.apache.cassandra.db.memtable.Flushing.createFlushWriter(Flushing.java:303)
		at org.apache.cassandra.db.memtable.Flushing.flushRunnable(Flushing.java:135)
		at org.apache.cassandra.db.memtable.Flushing.flushRunnables(Flushing.java:96)
		at org.apache.cassandra.db.memtable.Flushing.flushRunnables(Flushing.java:73)
		at org.apache.cassandra.db.ColumnFamilyStore$Flush.flushMemtable(ColumnFamilyStore.java:1360)
		at org.apache.cassandra.db.ColumnFamilyStore$Flush.run(ColumnFamilyStore.java:1315)

I think https://github.com/datastax/cassandra/pull/1255/commits/073452c45f4503f0971ac4b262dcb46de007e946 was a legitimate issue but https://github.com/datastax/cassandra/pull/1255/commits/43624fc6a385c4f70a368026530662f4f11ec65b seems a bit more questionable.

What are your thoughts?

michaeljmarshall avatar Sep 19 '24 15:09 michaeljmarshall

java.lang.UnsupportedOperationException: Token type BytesToken does not support token allocation. normally means the test is using ByteOrderedPartitioner. Since token allocation cannot work with it either, there's no point to try to fix the tests for that.

Actually we should not instantiate ShardManagerTokenAware if the partitioner for the table does not support splitting/sizing (!partitioner.splitter().isPresent()), and probably push its selection to after the test-specific adjustments here.

blambov avatar Sep 19 '24 16:09 blambov

If we do the above, neither of the two fixes above should be needed.

blambov avatar Sep 19 '24 16:09 blambov

@blambov - at this rate, UnifiedCompactionDensitiesTest is the only remaining compaction test that fails. Because I parameterized ShardedMultiWriterTest and added specific assertions about the distributions adding to about 1 and about the max number of tokens spanned, I think we have the spirit of the UnifiedCompactionDensitiesTest covered. I am going to disable DEFAULT_IS_NODE_AWARE now.

michaeljmarshall avatar Sep 24 '24 04:09 michaeljmarshall