RandomPartitioner has separate MessageDigest for token generation Patch by Sam Tunnicliffe; reviewed by Jason Brown for CASSANDRA-13964
Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/58daf137 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/58daf137 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/58daf137 Branch: refs/heads/trunk Commit: 58daf1376456289f97f0ef0b0daf9e0d03ba6b81 Parents: 6c29ee8 Author: Sam Tunnicliffe <s...@beobal.com> Authored: Tue Oct 17 14:51:43 2017 +0100 Committer: Sam Tunnicliffe <s...@beobal.com> Committed: Tue Nov 7 13:38:25 2017 +0000 ---------------------------------------------------------------------- CHANGES.txt | 1 + .../apache/cassandra/dht/RandomPartitioner.java | 43 ++++++++++++++++++-- .../org/apache/cassandra/utils/FBUtilities.java | 19 --------- 3 files changed, 41 insertions(+), 22 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/cassandra/blob/58daf137/CHANGES.txt ---------------------------------------------------------------------- diff --git a/CHANGES.txt b/CHANGES.txt index 935931c..3f4f3f2 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 3.0.16 + * Tracing interferes with digest requests when using RandomPartitioner (CASSANDRA-13964) * Add flag to disable materialized views, and warnings on creation (CASSANDRA-13959) * Don't let user drop or generally break tables in system_distributed (CASSANDRA-13813) * Provide a JMX call to sync schema with local storage (CASSANDRA-13954) http://git-wip-us.apache.org/repos/asf/cassandra/blob/58daf137/src/java/org/apache/cassandra/dht/RandomPartitioner.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/dht/RandomPartitioner.java b/src/java/org/apache/cassandra/dht/RandomPartitioner.java index b0dea01..c7837c9 100644 --- a/src/java/org/apache/cassandra/dht/RandomPartitioner.java +++ b/src/java/org/apache/cassandra/dht/RandomPartitioner.java @@ -20,6 +20,7 @@ package org.apache.cassandra.dht; import java.math.BigDecimal; import java.math.BigInteger; import java.nio.ByteBuffer; +import java.security.MessageDigest; import java.util.*; import com.google.common.annotations.VisibleForTesting; @@ -45,11 +46,35 @@ public class RandomPartitioner implements IPartitioner public static final BigIntegerToken MINIMUM = new BigIntegerToken("-1"); public static final BigInteger MAXIMUM = new BigInteger("2").pow(127); - private static final int HEAP_SIZE = (int) ObjectSizes.measureDeep(new BigIntegerToken(FBUtilities.hashToBigInteger(ByteBuffer.allocate(1)))); + /** + * Maintain a separate threadlocal message digest, exclusively for token hashing. This is necessary because + * when Tracing is enabled and using the default tracing implementation, creating the mutations for the trace + * events involves tokenizing the partition keys. This happens multiple times whilst servicing a ReadCommand, + * and so can interfere with the stateful digest calculation if the node is a replica producing a digest response. + */ + private static final ThreadLocal<MessageDigest> localMD5Digest = new ThreadLocal<MessageDigest>() + { + @Override + protected MessageDigest initialValue() + { + return FBUtilities.newMessageDigest("MD5"); + } + + @Override + public MessageDigest get() + { + MessageDigest digest = super.get(); + digest.reset(); + return digest; + } + }; + + private static final int HEAP_SIZE = (int) ObjectSizes.measureDeep(new BigIntegerToken(hashToBigInteger(ByteBuffer.allocate(1)))); public static final RandomPartitioner instance = new RandomPartitioner(); public static final AbstractType<?> partitionOrdering = new PartitionerDefinedOrder(instance); + public DecoratedKey decorateKey(ByteBuffer key) { return new CachedHashDecoratedKey(getToken(key), key); @@ -72,7 +97,7 @@ public class RandomPartitioner implements IPartitioner public BigIntegerToken getRandomToken() { - BigInteger token = FBUtilities.hashToBigInteger(GuidGenerator.guidAsBytes()); + BigInteger token = hashToBigInteger(GuidGenerator.guidAsBytes()); if ( token.signum() == -1 ) token = token.multiply(BigInteger.valueOf(-1L)); return new BigIntegerToken(token); @@ -160,7 +185,8 @@ public class RandomPartitioner implements IPartitioner { if (key.remaining() == 0) return MINIMUM; - return new BigIntegerToken(FBUtilities.hashToBigInteger(key)); + + return new BigIntegerToken(hashToBigInteger(key)); } public Map<Token, Float> describeOwnership(List<Token> sortedTokens) @@ -203,4 +229,15 @@ public class RandomPartitioner implements IPartitioner { return partitionOrdering; } + + private static BigInteger hashToBigInteger(ByteBuffer data) + { + MessageDigest messageDigest = localMD5Digest.get(); + if (data.hasArray()) + messageDigest.update(data.array(), data.arrayOffset() + data.position(), data.remaining()); + else + messageDigest.update(data.duplicate()); + + return new BigInteger(messageDigest.digest()).abs(); + } } http://git-wip-us.apache.org/repos/asf/cassandra/blob/58daf137/src/java/org/apache/cassandra/utils/FBUtilities.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/utils/FBUtilities.java b/src/java/org/apache/cassandra/utils/FBUtilities.java index 5562f5e..f111919 100644 --- a/src/java/org/apache/cassandra/utils/FBUtilities.java +++ b/src/java/org/apache/cassandra/utils/FBUtilities.java @@ -265,25 +265,6 @@ public class FBUtilities return out; } - public static byte[] hash(ByteBuffer... data) - { - MessageDigest messageDigest = localMD5Digest.get(); - for (ByteBuffer block : data) - { - if (block.hasArray()) - messageDigest.update(block.array(), block.arrayOffset() + block.position(), block.remaining()); - else - messageDigest.update(block.duplicate()); - } - - return messageDigest.digest(); - } - - public static BigInteger hashToBigInteger(ByteBuffer data) - { - return new BigInteger(hash(data)).abs(); - } - public static void sortSampledKeys(List<DecoratedKey> keys, Range<Token> range) { if (range.left.compareTo(range.right) >= 0) --------------------------------------------------------------------- To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org