Repository: cassandra Updated Branches: refs/heads/cassandra-2.0 ee160a971 -> da9f030c2 refs/heads/cassandra-2.1 ca6360704 -> 3b3570c8c refs/heads/trunk 4807f1295 -> 7f8a09840
Fix streaming throttle unit to bits patch by Björn Hegerfors; reviewed by yukim for CASSANDRA-8852 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/da9f030c Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/da9f030c Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/da9f030c Branch: refs/heads/cassandra-2.0 Commit: da9f030c2912748349d1874a544b97b9f08642d1 Parents: ee160a9 Author: Björn Hegerfors <[email protected]> Authored: Wed Feb 25 08:50:37 2015 -0800 Committer: Yuki Morishita <[email protected]> Committed: Wed Feb 25 08:52:14 2015 -0800 ---------------------------------------------------------------------- src/java/org/apache/cassandra/streaming/StreamManager.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/cassandra/blob/da9f030c/src/java/org/apache/cassandra/streaming/StreamManager.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/streaming/StreamManager.java b/src/java/org/apache/cassandra/streaming/StreamManager.java index b5b1c7f..4871078 100644 --- a/src/java/org/apache/cassandra/streaming/StreamManager.java +++ b/src/java/org/apache/cassandra/streaming/StreamManager.java @@ -63,17 +63,17 @@ public class StreamManager implements StreamManagerMBean public static class StreamRateLimiter { - private static final double ONE_MEGA_BIT = 1024 * 1024 * 8; + private static final double BYTES_PER_MEGABIT = 1024 * 1024 / 8; private static final RateLimiter limiter = RateLimiter.create(Double.MAX_VALUE); private static final RateLimiter interDCLimiter = RateLimiter.create(Double.MAX_VALUE); private final boolean isLocalDC; public StreamRateLimiter(InetAddress peer) { - double throughput = ((double) DatabaseDescriptor.getStreamThroughputOutboundMegabitsPerSec()) * ONE_MEGA_BIT; + double throughput = ((double) DatabaseDescriptor.getStreamThroughputOutboundMegabitsPerSec()) * BYTES_PER_MEGABIT; mayUpdateThroughput(throughput, limiter); - double interDCThroughput = ((double) DatabaseDescriptor.getInterDCStreamThroughputOutboundMegabitsPerSec()) * ONE_MEGA_BIT; + double interDCThroughput = ((double) DatabaseDescriptor.getInterDCStreamThroughputOutboundMegabitsPerSec()) * BYTES_PER_MEGABIT; mayUpdateThroughput(interDCThroughput, interDCLimiter); if (DatabaseDescriptor.getLocalDataCenter() != null && DatabaseDescriptor.getEndpointSnitch() != null)
