Updated Branches: refs/heads/cassandra-1.1 cbac7af79 -> e36b2130f
Merge branch 'cassandra-1.0' into cassandra-1.1 Conflicts: NEWS.txt src/java/org/apache/cassandra/config/DatabaseDescriptor.java src/java/org/apache/cassandra/net/OutboundTcpConnectionPool.java src/java/org/apache/cassandra/service/MigrationManager.java Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/e36b2130 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/e36b2130 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/e36b2130 Branch: refs/heads/cassandra-1.1 Commit: e36b2130ff2123fa1ebc3627d026407956611d84 Parents: cbac7af a35f878 Author: Sylvain Lebresne <sylv...@riptano.com> Authored: Sun Feb 5 22:38:45 2012 +0100 Committer: Sylvain Lebresne <sylv...@riptano.com> Committed: Sun Feb 5 22:38:45 2012 +0100 ---------------------------------------------------------------------- CHANGES.txt | 1 + NEWS.txt | 8 ++++++++ conf/cassandra.yaml | 7 +++++++ src/java/org/apache/cassandra/config/Config.java | 2 ++ .../cassandra/config/DatabaseDescriptor.java | 5 +++++ .../cassandra/net/OutboundTcpConnection.java | 5 +++++ .../cassandra/net/OutboundTcpConnectionPool.java | 2 +- .../apache/cassandra/streaming/FileStreamTask.java | 1 + .../cassandra/streaming/IncomingStreamReader.java | 1 + 9 files changed, 31 insertions(+), 1 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/cassandra/blob/e36b2130/CHANGES.txt ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/cassandra/blob/e36b2130/NEWS.txt ---------------------------------------------------------------------- diff --cc NEWS.txt index 8fca1cd,e36b9e8..fc401b8 --- a/NEWS.txt +++ b/NEWS.txt @@@ -8,69 -8,13 +8,77 @@@ upgrade, just in case you need to roll (Cassandra version X + 1 will always be able to read data files created by version X, but the inverse is not necessarily the case.) + +1.1 +=== + +Upgrading +--------- + - The KsDef.replication_factor field (deprecated since 0.8) has + been removed. Older clients will need to be updated to be able + to continue to created and update keyspaces. + - If you are running a multi datacenter setup, you should upgrade to + the latest 1.0.x (or 0.8.x) release before upgrading. Versions + 0.8.8 and 1.0.3-1.0.5 generate cross-dc forwarding that is incompatible + with 1.1. + - EACH_QUORUM ConsistencyLevel is only supported for writes and will now + throw an InvalidRequestException when used for reads. (Previous + versions would silently perform a LOCAL_QUORUM read instead.) + - ANY ConsistencyLevel is only supported for writes and will now + throw an InvalidRequestException when used for reads. (Previous + versions would silently perform a ONE read for range queries; + single-row and multiget reads already rejected ANY.) + - The largest mutation batch accepted by the commitlog is now 128MB. + (In practice, batches larger than ~10MB always caused poor + performance due to load volatility and GC promotion failures.) + Larger batches will continue to be accepted but will not be + durable. Consider setting durable_writes=false if you really + want to use such large batches. + - Make sure that global settings: key_cache_{size_in_mb, save_period} + and row_cache_{size_in_mb, save_period} in conf/cassandra.yaml are + used instead of per-ColumnFamily options. + - JMX methods no longer return custom Cassandra objects. Any such methods + will now return standard Maps, Lists, etc. + - Hadoop input and output details are now separated. If you were + previously using methods such as getRpcPort you now need to use + getInputRpcPort or getOutputRpcPort depending on the circumstance. + - CQL changes: + + Prior to 1.1, you could use KEY as the primary key name in some + select statements, even if the PK was actually given a different + name. In 1.1+ you must use the defined PK name. + + +Features +-------- + - Cassandra 1.1 adds row-level isolation. Multi-column updates to + a single row have always been *atomic* (either all will be applied, + or none) thanks to the CommitLog, but until 1.1 they were not *isolated* + -- a reader may see mixed old and new values while the update happens. + - Finer-grained control over data directories, allowing a ColumnFamily to + be pinned to specfic media. + - Hadoop: a new BulkOutputFormat is included which will directly write + SSTables locally and then stream them into the cluster. + - Hadoop: KeyRange.filter is now supported with ColumnFamilyInputFormat + - The bulk loader is not longer a fat client; it can be run from an + existing machine in a cluster. + - A new write survey mode has been added, similar to bootstrap (enabled via + -Dcassandra.write_survey=true), but the node will not automatically join + the cluster. This is useful for cases such as testing different + compaction strategies with live traffic without affecting the cluster. + - Key and row caches are now global, similar to the global memtable + threshold. + - Off-heap caches no longer require JNA. + - Streaming is now multithreaded. + + + 1.0.8 + ===== + + Other + ----- + - Allow configuring socket timeout for streaming + ++ 1.0.7 ===== http://git-wip-us.apache.org/repos/asf/cassandra/blob/e36b2130/conf/cassandra.yaml ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/cassandra/blob/e36b2130/src/java/org/apache/cassandra/config/Config.java ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/cassandra/blob/e36b2130/src/java/org/apache/cassandra/config/DatabaseDescriptor.java ---------------------------------------------------------------------- diff --cc src/java/org/apache/cassandra/config/DatabaseDescriptor.java index e301308,5aa59e4..fdff2a7 --- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java +++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java @@@ -983,38 -1009,8 +983,43 @@@ public class DatabaseDescripto return conf.commitlog_total_space_in_mb; } + public static int getKeyCacheSizeInMB() + { + return conf.key_cache_size_in_mb; + } + + public static int getKeyCacheSavePeriod() + { + return conf.key_cache_save_period; + } + + public static int getKeyCacheKeysToSave() + { + return conf.key_cache_keys_to_save; + } + + public static int getRowCacheSizeInMB() + { + return conf.row_cache_size_in_mb; + } + + public static int getRowCacheSavePeriod() + { + return conf.row_cache_save_period; + } + + public static int getRowCacheKeysToSave() + { + return conf.row_cache_keys_to_save; + } + + public static IRowCacheProvider getRowCacheProvider() + { + return rowCacheProvider; + } ++ + public static int getStreamingSocketTimeout() + { + return conf.streaming_socket_timeout_in_ms; + } } http://git-wip-us.apache.org/repos/asf/cassandra/blob/e36b2130/src/java/org/apache/cassandra/net/OutboundTcpConnection.java ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/cassandra/blob/e36b2130/src/java/org/apache/cassandra/net/OutboundTcpConnectionPool.java ---------------------------------------------------------------------- diff --cc src/java/org/apache/cassandra/net/OutboundTcpConnectionPool.java index 1bf28b5,a75dafe..ed820b6 --- a/src/java/org/apache/cassandra/net/OutboundTcpConnectionPool.java +++ b/src/java/org/apache/cassandra/net/OutboundTcpConnectionPool.java @@@ -78,17 -77,11 +78,17 @@@ public class OutboundTcpConnectionPoo // zero means 'bind on any available port.' if (isEncryptedChannel()) { - return SSLFactory.getSocket(DatabaseDescriptor.getEncryptionOptions(), endPoint(), DatabaseDescriptor.getSSLStoragePort(), FBUtilities.getLocalAddress(), 0); + if (Config.getOutboundBindAny()) + return SSLFactory.getSocket(DatabaseDescriptor.getEncryptionOptions(), endPoint(), DatabaseDescriptor.getSSLStoragePort()); + else + return SSLFactory.getSocket(DatabaseDescriptor.getEncryptionOptions(), endPoint(), DatabaseDescriptor.getSSLStoragePort(), FBUtilities.getLocalAddress(), 0); } -- else ++ else { - return new Socket(endPoint(), DatabaseDescriptor.getStoragePort(), FBUtilities.getLocalAddress(), 0); + if (Config.getOutboundBindAny()) + return new Socket(endPoint(), DatabaseDescriptor.getStoragePort()); + else + return new Socket(endPoint(), DatabaseDescriptor.getStoragePort(), FBUtilities.getLocalAddress(), 0); } } http://git-wip-us.apache.org/repos/asf/cassandra/blob/e36b2130/src/java/org/apache/cassandra/streaming/FileStreamTask.java ---------------------------------------------------------------------- diff --cc src/java/org/apache/cassandra/streaming/FileStreamTask.java index e04b9b9,ffb1388..8cd9a63 --- a/src/java/org/apache/cassandra/streaming/FileStreamTask.java +++ b/src/java/org/apache/cassandra/streaming/FileStreamTask.java @@@ -232,8 -197,8 +232,9 @@@ public class FileStreamTask extends Wra try { socket = MessagingService.instance().getConnectionPool(to).newSocket(); + socket.setSoTimeout(DatabaseDescriptor.getStreamingSocketTimeout()); output = socket.getOutputStream(); + input = new DataInputStream(socket.getInputStream()); break; } catch (IOException e) http://git-wip-us.apache.org/repos/asf/cassandra/blob/e36b2130/src/java/org/apache/cassandra/streaming/IncomingStreamReader.java ---------------------------------------------------------------------- diff --cc src/java/org/apache/cassandra/streaming/IncomingStreamReader.java index c97c451,8ade06a..f57b400 --- a/src/java/org/apache/cassandra/streaming/IncomingStreamReader.java +++ b/src/java/org/apache/cassandra/streaming/IncomingStreamReader.java @@@ -56,12 -55,10 +56,13 @@@ public class IncomingStreamReade public IncomingStreamReader(StreamHeader header, Socket socket) throws IOException { + socket.setSoTimeout(DatabaseDescriptor.getStreamingSocketTimeout()); this.socket = socket; - InetSocketAddress remoteAddress = (InetSocketAddress)socket.getRemoteSocketAddress(); - session = StreamInSession.get(remoteAddress.getAddress(), header.sessionId); + InetAddress host = header.broadcastAddress != null ? header.broadcastAddress + : ((InetSocketAddress)socket.getRemoteSocketAddress()).getAddress(); + session = StreamInSession.get(host, header.sessionId); + session.setSocket(socket); + session.addFiles(header.pendingFiles); // set the current file we are streaming so progress shows up in jmx session.setCurrentFile(header.file);