fix CME in SessionInfo updateProgress affecting netstats patch by Josh McKenzie; reviewed by Tyler Hobbs for CASSANDRA-6577
Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/468cc72f Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/468cc72f Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/468cc72f Branch: refs/heads/trunk Commit: 468cc72face6593effe542ab45b48e99b436f79a Parents: 50e544f Author: Jonathan Ellis <[email protected]> Authored: Sat Feb 22 08:15:32 2014 -0600 Committer: Jonathan Ellis <[email protected]> Committed: Sat Feb 22 08:15:40 2014 -0600 ---------------------------------------------------------------------- CHANGES.txt | 1 + src/java/org/apache/cassandra/streaming/SessionInfo.java | 8 ++++---- 2 files changed, 5 insertions(+), 4 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/cassandra/blob/468cc72f/CHANGES.txt ---------------------------------------------------------------------- diff --git a/CHANGES.txt b/CHANGES.txt index b25ff47..bfcb6a4 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 2.0.6 + * fix CME in SessionInfo updateProgress affecting netstats (CASSANDRA-6577) * Allow repairing between specific replicas (CASSANDRA-6440) * Allow per-dc enabling of hints (CASSANDRA-6157) * Add compatibility for Hadoop 0.2.x (CASSANDRA-5201) http://git-wip-us.apache.org/repos/asf/cassandra/blob/468cc72f/src/java/org/apache/cassandra/streaming/SessionInfo.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/streaming/SessionInfo.java b/src/java/org/apache/cassandra/streaming/SessionInfo.java index 9532041..b722ecf 100644 --- a/src/java/org/apache/cassandra/streaming/SessionInfo.java +++ b/src/java/org/apache/cassandra/streaming/SessionInfo.java @@ -20,8 +20,8 @@ package org.apache.cassandra.streaming; import java.io.Serializable; import java.net.InetAddress; import java.util.Collection; -import java.util.HashMap; import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; import com.google.common.base.Predicate; import com.google.common.collect.ImmutableSet; @@ -51,8 +51,8 @@ public final class SessionInfo implements Serializable this.peer = peer; this.receivingSummaries = ImmutableSet.copyOf(receivingSummaries); this.sendingSummaries = ImmutableSet.copyOf(sendingSummaries); - this.receivingFiles = new HashMap<>(); - this.sendingFiles = new HashMap<>(); + this.receivingFiles = new ConcurrentHashMap<>(); + this.sendingFiles = new ConcurrentHashMap<>(); this.state = state; } @@ -66,7 +66,7 @@ public final class SessionInfo implements Serializable * * @param newProgress new progress info */ - public synchronized void updateProgress(ProgressInfo newProgress) + public void updateProgress(ProgressInfo newProgress) { assert peer.equals(newProgress.peer);
