cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From yu...@apache.org
Subject [3/3] cassandra git commit: Merge branch 'cassandra-2.1' into cassandra-2.2
Date Thu, 30 Jul 2015 00:57:12 GMT
Merge branch 'cassandra-2.1' into cassandra-2.2


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/ab6e4854
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/ab6e4854
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/ab6e4854

Branch: refs/heads/cassandra-2.2
Commit: ab6e4854d4b83abfd9d46479a24d22f4585ef63b
Parents: 2da0169 0a4728f
Author: Yuki Morishita <yukim@apache.org>
Authored: Wed Jul 29 19:55:35 2015 -0500
Committer: Yuki Morishita <yukim@apache.org>
Committed: Wed Jul 29 19:55:35 2015 -0500

----------------------------------------------------------------------
 CHANGES.txt                                                 | 3 +++
 src/java/org/apache/cassandra/streaming/StreamSession.java  | 9 ++++++---
 .../org/apache/cassandra/thrift/ThriftSessionManager.java   | 9 ++++++---
 3 files changed, 15 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/ab6e4854/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 50ac561,5cfc347..28e6c4a
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,23 -1,7 +1,26 @@@
 -2.1.9
 +2.2.1
 + * UDF / UDA execution time in trace (CASSANDRA-9723)
 + * Remove repair snapshot leftover on startup (CASSANDRA-7357)
 + * Use random nodes for batch log when only 2 racks (CASSANDRA-8735)
 + * Fix broken internode SSL (CASSANDRA-9884)
 +Merged from 2.1:
   * Fix handling of enable/disable autocompaction (CASSANDRA-9899)
 - * Commit log segment recycling is disabled by default (CASSANDRA-9896)
   * Add consistency level to tracing ouput (CASSANDRA-9827)
++ * Remove repair snapshot leftover on startup (CASSANDRA-7357)
++ * Use random nodes for batch log when only 2 racks (CASSANDRA-8735)
++ * Ensure atomicity inside thrift and stream session (CASSANDRA-7757)
 +Merged from 2.0:
 + * Log when messages are dropped due to cross_node_timeout (CASSANDRA-9793)
 + * Don't track hotness when opening from snapshot for validation (CASSANDRA-9382)
 +
 +
 +2.2.0
 + * Allow the selection of columns together with aggregates (CASSANDRA-9767)
 + * Fix cqlsh copy methods and other windows specific issues (CASSANDRA-9795)
 + * Don't wrap byte arrays in SequentialWriter (CASSANDRA-9797)
 + * sum() and avg() functions missing for smallint and tinyint types (CASSANDRA-9671)
 + * Revert CASSANDRA-9542 (allow native functions in UDA) (CASSANDRA-9771)
 +Merged from 2.1:
   * Fix MarshalException when upgrading superColumn family (CASSANDRA-9582)
   * Fix broken logging for "empty" flushes in Memtable (CASSANDRA-9837)
   * Handle corrupt files on startup (CASSANDRA-9686)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/ab6e4854/src/java/org/apache/cassandra/streaming/StreamSession.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/streaming/StreamSession.java
index 7236194,63219d8..366bc33
--- a/src/java/org/apache/cassandra/streaming/StreamSession.java
+++ b/src/java/org/apache/cassandra/streaming/StreamSession.java
@@@ -135,9 -135,9 +135,9 @@@ public class StreamSession implements I
      private StreamResultFuture streamResult;
  
      // stream requests to send to the peer
 -    private final Set<StreamRequest> requests = Sets.newConcurrentHashSet();
 +    protected final Set<StreamRequest> requests = Sets.newConcurrentHashSet();
      // streaming tasks are created and managed per ColumnFamily ID
-     private final Map<UUID, StreamTransferTask> transfers = new ConcurrentHashMap<>();
+     private final ConcurrentHashMap<UUID, StreamTransferTask> transfers = new ConcurrentHashMap<>();
      // data receivers, filled after receiving prepare message
      private final Map<UUID, StreamReceiveTask> receivers = new ConcurrentHashMap<>();
      private final StreamingMetrics metrics;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/ab6e4854/src/java/org/apache/cassandra/thrift/ThriftSessionManager.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/thrift/ThriftSessionManager.java
index 2703b52,6caa558..3603ad5
--- a/src/java/org/apache/cassandra/thrift/ThriftSessionManager.java
+++ b/src/java/org/apache/cassandra/thrift/ThriftSessionManager.java
@@@ -58,8 -57,11 +58,11 @@@ public class ThriftSessionManage
          ThriftClientState cState = activeSocketSessions.get(socket);
          if (cState == null)
          {
-             cState = new ThriftClientState((InetSocketAddress)socket);
-             activeSocketSessions.put(socket, cState);
+             //guarantee atomicity
 -            ThriftClientState newState = new ThriftClientState(socket);
++            ThriftClientState newState = new ThriftClientState((InetSocketAddress)socket);
+             cState = activeSocketSessions.putIfAbsent(socket, newState);
+             if (cState == null)
+                 cState = newState;
          }
          return cState;
      }


Mime
View raw message