merge from 1.1
Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/40669a33 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/40669a33 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/40669a33 Branch: refs/heads/cassandra-1.2 Commit: 40669a33027d241ceda76e842afb5de0bcada5b6 Parents: b69c1aa ac19c12 Author: Jonathan Ellis <[email protected]> Authored: Tue May 21 13:41:46 2013 -0500 Committer: Jonathan Ellis <[email protected]> Committed: Tue May 21 13:41:46 2013 -0500 ---------------------------------------------------------------------- CHANGES.txt | 11 ++++++++ conf/cassandra.yaml | 6 +---- src/java/org/apache/cassandra/config/Config.java | 2 + .../cassandra/config/DatabaseDescriptor.java | 8 ------ .../cassandra/hadoop/ColumnFamilyOutputFormat.java | 4 +- .../cassandra/hadoop/ColumnFamilyRecordReader.java | 2 +- .../org/apache/cassandra/hadoop/ConfigHelper.java | 15 ++--------- .../apache/cassandra/thrift/TBinaryProtocol.java | 19 --------------- .../org/apache/cassandra/thrift/ThriftServer.java | 2 +- 9 files changed, 21 insertions(+), 48 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/cassandra/blob/40669a33/CHANGES.txt ---------------------------------------------------------------------- diff --cc CHANGES.txt index 619e415,501a68f..103f659 --- a/CHANGES.txt +++ b/CHANGES.txt @@@ -1,113 -1,14 +1,124 @@@ -1.1.12 +1.2.6 + * Write row markers when serializing schema (CASSANDRA-5572) + * Check only SSTables for the requested range when streaming (CASSANDRA-5569) ++Merged from 1.1: + * Remove buggy thrift max message length option (CASSANDRA-5529) + + +1.2.5 + * make BytesToken.toString only return hex bytes (CASSANDRA-5566) + * Ensure that submitBackground enqueues at least one task (CASSANDRA-5554) + * fix 2i updates with identical values and timestamps (CASSANDRA-5540) + * fix compaction throttling bursty-ness (CASSANDRA-4316) + * reduce memory consumption of IndexSummary (CASSANDRA-5506) + * remove per-row column name bloom filters (CASSANDRA-5492) + * Include fatal errors in trace events (CASSANDRA-5447) + * Ensure that PerRowSecondaryIndex is notified of row-level deletes + (CASSANDRA-5445) + * Allow empty blob literals in CQL3 (CASSANDRA-5452) + * Fix streaming RangeTombstones at column index boundary (CASSANDRA-5418) + * Fix preparing statements when current keyspace is not set (CASSANDRA-5468) + * Fix SemanticVersion.isSupportedBy minor/patch handling (CASSANDRA-5496) + * Don't provide oldCfId for post-1.1 system cfs (CASSANDRA-5490) + * Fix primary range ignores replication strategy (CASSANDRA-5424) + * Fix shutdown of binary protocol server (CASSANDRA-5507) + * Fix repair -snapshot not working (CASSANDRA-5512) + * Set isRunning flag later in binary protocol server (CASSANDRA-5467) + * Fix use of CQL3 functions with descending clustering order (CASSANDRA-5472) + * Disallow renaming columns one at a time for thrift table in CQL3 + (CASSANDRA-5531) + * cqlsh: add CLUSTERING ORDER BY support to DESCRIBE (CASSANDRA-5528) + * Add custom secondary index support to CQL3 (CASSANDRA-5484) + * Fix repair hanging silently on unexpected error (CASSANDRA-5229) + * Fix Ec2Snitch regression introduced by CASSANDRA-5171 (CASSANDRA-5432) + * Add nodetool enablebackup/disablebackup (CASSANDRA-5556) + * cqlsh: fix DESCRIBE after case insensitive USE (CASSANDRA-5567) +Merged from 1.1 * Add retry mechanism to OTC for non-droppable_verbs (CASSANDRA-5393) - * Use allocator information to improve memtable memory usage estimate + * Use allocator information to improve memtable memory usage estimate (CASSANDRA-5497) + * Fix trying to load deleted row into row cache on startup (CASSANDRA-4463) * fsync leveled manifest to avoid corruption (CASSANDRA-5535) * Fix Bound intersection computation (CASSANDRA-5551) + * sstablescrub now respects max memory size in cassandra.in.sh (CASSANDRA-5562) + + +1.2.4 + * Ensure that PerRowSecondaryIndex updates see the most recent values + (CASSANDRA-5397) + * avoid duplicate index entries ind PrecompactedRow and + ParallelCompactionIterable (CASSANDRA-5395) + * remove the index entry on oldColumn when new column is a tombstone + (CASSANDRA-5395) + * Change default stream throughput from 400 to 200 mbps (CASSANDRA-5036) + * Gossiper logs DOWN for symmetry with UP (CASSANDRA-5187) + * Fix mixing prepared statements between keyspaces (CASSANDRA-5352) + * Fix consistency level during bootstrap - strike 3 (CASSANDRA-5354) + * Fix transposed arguments in AlreadyExistsException (CASSANDRA-5362) + * Improve asynchronous hint delivery (CASSANDRA-5179) + * Fix Guava dependency version (12.0 -> 13.0.1) for Maven (CASSANDRA-5364) + * Validate that provided CQL3 collection value are < 64K (CASSANDRA-5355) + * Make upgradeSSTable skip current version sstables by default (CASSANDRA-5366) + * Optimize min/max timestamp collection (CASSANDRA-5373) + * Invalid streamId in cql binary protocol when using invalid CL + (CASSANDRA-5164) + * Fix validation for IN where clauses with collections (CASSANDRA-5376) + * Copy resultSet on count query to avoid ConcurrentModificationException + (CASSANDRA-5382) + * Correctly typecheck in CQL3 even with ReversedType (CASSANDRA-5386) + * Fix streaming compressed files when using encryption (CASSANDRA-5391) + * cassandra-all 1.2.0 pom missing netty dependency (CASSANDRA-5392) + * Fix writetime/ttl functions on null values (CASSANDRA-5341) + * Fix NPE during cql3 select with token() (CASSANDRA-5404) + * IndexHelper.skipBloomFilters won't skip non-SHA filters (CASSANDRA-5385) + * cqlsh: Print maps ordered by key, sort sets (CASSANDRA-5413) + * Add null syntax support in CQL3 for inserts (CASSANDRA-3783) + * Allow unauthenticated set_keyspace() calls (CASSANDRA-5423) + * Fix potential incremental backups race (CASSANDRA-5410) + * Fix prepared BATCH statements with batch-level timestamps (CASSANDRA-5415) + * Allow overriding superuser setup delay (CASSANDRA-5430) + * cassandra-shuffle with JMX usernames and passwords (CASSANDRA-5431) +Merged from 1.1: + * cli: Quote ks and cf names in schema output when needed (CASSANDRA-5052) + * Fix bad default for min/max timestamp in SSTableMetadata (CASSANDRA-5372) + * Fix cf name extraction from manifest in Directories.migrateFile() + (CASSANDRA-5242) + * Support pluggable internode authentication (CASSANDRA-5401) ++<<<<<<< HEAD + +1.2.3 + * add check for sstable overlap within a level on startup (CASSANDRA-5327) + * replace ipv6 colons in jmx object names (CASSANDRA-5298, 5328) + * Avoid allocating SSTableBoundedScanner during repair when the range does + not intersect the sstable (CASSANDRA-5249) + * Don't lowercase property map keys (this breaks NTS) (CASSANDRA-5292) + * Fix composite comparator with super columns (CASSANDRA-5287) + * Fix insufficient validation of UPDATE queries against counter cfs + (CASSANDRA-5300) + * Fix PropertyFileSnitch default DC/Rack behavior (CASSANDRA-5285) + * Handle null values when executing prepared statement (CASSANDRA-5081) + * Add netty to pom dependencies (CASSANDRA-5181) + * Include type arguments in Thrift CQLPreparedResult (CASSANDRA-5311) + * Fix compaction not removing columns when bf_fp_ratio is 1 (CASSANDRA-5182) + * cli: Warn about missing CQL3 tables in schema descriptions (CASSANDRA-5309) + * Re-enable unknown option in replication/compaction strategies option for + backward compatibility (CASSANDRA-4795) + * Add binary protocol support to stress (CASSANDRA-4993) + * cqlsh: Fix COPY FROM value quoting and null handling (CASSANDRA-5305) + * Fix repair -pr for vnodes (CASSANDRA-5329) + * Relax CL for auth queries for non-default users (CASSANDRA-5310) + * Fix AssertionError during repair (CASSANDRA-5245) + * Don't announce migrations to pre-1.2 nodes (CASSANDRA-5334) +Merged from 1.1: ++||||||| merged common ancestors ++1.1.11 ++ * Fix trying to load deleted row into row cache on startup (CASSANDRA-4463) ++======= + + 1.1.11 + * Fix trying to load deleted row into row cache on startup (CASSANDRA-4463) ++>>>>>>> cassandra-1.1 * Update offline scrub for 1.0 -> 1.1 directory structure (CASSANDRA-5195) * add tmp flag to Descriptor hashcode (CASSANDRA-4021) * fix logging of "Found table data in data directories" when only system tables http://git-wip-us.apache.org/repos/asf/cassandra/blob/40669a33/conf/cassandra.yaml ---------------------------------------------------------------------- diff --cc conf/cassandra.yaml index cf7b139,027479d..4d2eff3 --- a/conf/cassandra.yaml +++ b/conf/cassandra.yaml @@@ -393,28 -330,14 +393,24 @@@ rpc_server_type: syn # rpc_send_buff_size_in_bytes: # rpc_recv_buff_size_in_bytes: +# Uncomment to set socket buffer size for internode communication +# Note that when setting this, the buffer size is limited by net.core.wmem_max +# and when not setting it it is defined by net.ipv4.tcp_wmem +# See: +# /proc/sys/net/core/wmem_max +# /proc/sys/net/core/rmem_max +# /proc/sys/net/ipv4/tcp_wmem +# /proc/sys/net/ipv4/tcp_wmem +# and: man tcp +# internode_send_buff_size_in_bytes: +# internode_recv_buff_size_in_bytes: + - # Frame size for thrift (maximum field length). + # Frame size for thrift (maximum message length). -# 0 disables TFramedTransport in favor of TSocket. This option -# is deprecated; we strongly recommend using Framed mode. thrift_framed_transport_size_in_mb: 15 - # The max length of a thrift message, including all fields and - # internal thrift overhead. - thrift_max_message_length_in_mb: 16 - # Set to true to have Cassandra create a hard link to each sstable # flushed or streamed locally in a backups/ subdirectory of the -# Keyspace data. Removing these links is the operator's +# keyspace data. Removing these links is the operator's # responsibility. incremental_backups: false http://git-wip-us.apache.org/repos/asf/cassandra/blob/40669a33/src/java/org/apache/cassandra/config/Config.java ---------------------------------------------------------------------- diff --cc src/java/org/apache/cassandra/config/Config.java index ad99809,11beea6..7fbd019 --- a/src/java/org/apache/cassandra/config/Config.java +++ b/src/java/org/apache/cassandra/config/Config.java @@@ -85,19 -72,14 +85,21 @@@ public class Confi public Integer rpc_port = 9160; public String rpc_server_type = "sync"; public Boolean rpc_keepalive = true; - public Integer rpc_min_threads = null; - public Integer rpc_max_threads = null; + public Integer rpc_min_threads = 16; + public Integer rpc_max_threads = Integer.MAX_VALUE; public Integer rpc_send_buff_size_in_bytes; public Integer rpc_recv_buff_size_in_bytes; + public Integer internode_send_buff_size_in_bytes; + public Integer internode_recv_buff_size_in_bytes; + + public Boolean start_native_transport = false; + public Integer native_transport_port = 9042; + public Integer native_transport_min_threads = 16; + public Integer native_transport_max_threads = 128; + @Deprecated public Integer thrift_max_message_length_in_mb = 16; + public Integer thrift_framed_transport_size_in_mb = 15; public Boolean snapshot_before_compaction = false; public Boolean auto_snapshot = true; http://git-wip-us.apache.org/repos/asf/cassandra/blob/40669a33/src/java/org/apache/cassandra/config/DatabaseDescriptor.java ---------------------------------------------------------------------- diff --cc src/java/org/apache/cassandra/config/DatabaseDescriptor.java index 006dfa1,f55c89a..a5bfcf2 --- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java +++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java @@@ -595,21 -572,11 +592,16 @@@ public class DatabaseDescripto return authenticator; } - public static IAuthority getAuthority() + public static IAuthorizer getAuthorizer() { - return authority; + return authorizer; + } + + public static int getPermissionsValidity() + { + return conf.permissions_validity_in_ms; } - public static int getThriftMaxMessageLength() - { - return conf.thrift_max_message_length_in_mb * 1024 * 1024; - } - public static int getThriftFramedTransportSize() { return conf.thrift_framed_transport_size_in_mb * 1024 * 1024; http://git-wip-us.apache.org/repos/asf/cassandra/blob/40669a33/src/java/org/apache/cassandra/hadoop/ColumnFamilyOutputFormat.java ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/cassandra/blob/40669a33/src/java/org/apache/cassandra/hadoop/ColumnFamilyRecordReader.java ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/cassandra/blob/40669a33/src/java/org/apache/cassandra/hadoop/ConfigHelper.java ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/cassandra/blob/40669a33/src/java/org/apache/cassandra/thrift/TBinaryProtocol.java ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/cassandra/blob/40669a33/src/java/org/apache/cassandra/thrift/ThriftServer.java ---------------------------------------------------------------------- diff --cc src/java/org/apache/cassandra/thrift/ThriftServer.java index b1c2f9c,0000000..0e2663a mode 100644,000000..100644 --- a/src/java/org/apache/cassandra/thrift/ThriftServer.java +++ b/src/java/org/apache/cassandra/thrift/ThriftServer.java @@@ -1,120 -1,0 +1,120 @@@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.thrift; + +import java.net.InetAddress; +import java.net.InetSocketAddress; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.service.CassandraDaemon; +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.thrift.server.TServer; +import org.apache.thrift.transport.TFramedTransport; + +public class ThriftServer implements CassandraDaemon.Server +{ + private static Logger logger = LoggerFactory.getLogger(ThriftServer.class); + final static String SYNC = "sync"; + final static String ASYNC = "async"; + final static String HSHA = "hsha"; + + private final InetAddress address; + private final int port; + private volatile ThriftServerThread server; + + public ThriftServer(InetAddress address, int port) + { + this.address = address; + this.port = port; + } + + public void start() + { + if (server == null) + { + server = new ThriftServerThread(address, port); + server.start(); + } + } + + public void stop() + { + if (server != null) + { + server.stopServer(); + try + { + server.join(); + } + catch (InterruptedException e) + { + logger.error("Interrupted while waiting thrift server to stop", e); + } + server = null; + } + } + + public boolean isRunning() + { + return server != null; + } + + /** + * Simple class to run the thrift connection accepting code in separate + * thread of control. + */ + private static class ThriftServerThread extends Thread + { + private TServer serverEngine; + + public ThriftServerThread(InetAddress listenAddr, int listenPort) + { + // now we start listening for clients + logger.info(String.format("Binding thrift service to %s:%s", listenAddr, listenPort)); + + TServerFactory.Args args = new TServerFactory.Args(); - args.tProtocolFactory = new TBinaryProtocol.Factory(true, true, DatabaseDescriptor.getThriftMaxMessageLength()); ++ args.tProtocolFactory = new TBinaryProtocol.Factory(true, true); + args.addr = new InetSocketAddress(listenAddr, listenPort); + args.cassandraServer = new CassandraServer(); + args.processor = new Cassandra.Processor(args.cassandraServer); + args.keepAlive = DatabaseDescriptor.getRpcKeepAlive(); + args.sendBufferSize = DatabaseDescriptor.getRpcSendBufferSize(); + args.recvBufferSize = DatabaseDescriptor.getRpcRecvBufferSize(); + int tFramedTransportSize = DatabaseDescriptor.getThriftFramedTransportSize(); + + logger.info("Using TFramedTransport with a max frame size of {} bytes.", tFramedTransportSize); + args.inTransportFactory = new TFramedTransport.Factory(tFramedTransportSize); + args.outTransportFactory = new TFramedTransport.Factory(tFramedTransportSize); + serverEngine = new TServerCustomFactory(DatabaseDescriptor.getRpcServerType()).buildTServer(args); + } + + public void run() + { + logger.info("Listening for thrift clients..."); + serverEngine.serve(); + } + + public void stopServer() + { + logger.info("Stop listening to thrift clients"); + serverEngine.stop(); + } + } +}
